diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java index 9bad2e3486e7f..a6a7e18b1f6ab 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java @@ -18,8 +18,6 @@ package org.apache.hudi.client; -import com.codahale.metrics.Timer; -import org.apache.hadoop.conf.Configuration; import org.apache.hudi.async.AsyncArchiveService; import org.apache.hudi.async.AsyncCleanerService; import org.apache.hudi.avro.model.HoodieCleanMetadata; @@ -78,6 +76,9 @@ import org.apache.hudi.table.marker.WriteMarkersFactory; import org.apache.hudi.table.upgrade.SupportsUpgradeDowngrade; import org.apache.hudi.table.upgrade.UpgradeDowngrade; + +import com.codahale.metrics.Timer; +import org.apache.hadoop.conf.Configuration; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -242,11 +243,11 @@ protected void commit(HoodieTable table, String commitActionType, String instant Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); } - protected HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf) { + protected HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf) { return createTable(config, hadoopConf, false); } - protected abstract HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf, boolean refreshTimeline); + protected abstract HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf, boolean refreshTimeline); void emitCommitMetrics(String instantTime, HoodieCommitMetadata metadata, String actionType) { try { @@ -397,7 +398,7 @@ protected void rollbackFailedBootstrap() { * @return Collection of WriteStatus to inspect errors and counts */ public abstract O bulkInsert(I records, final String instantTime, - Option> userDefinedBulkInsertPartitioner); + Option userDefinedBulkInsertPartitioner); /** @@ -417,7 +418,7 @@ public abstract O bulkInsert(I records, final String instantTime, * @return Collection of WriteStatus to inspect errors and counts */ public abstract O bulkInsertPreppedRecords(I preppedRecords, final String instantTime, - Option> bulkInsertPartitioner); + Option bulkInsertPartitioner); /** * Deletes a list of {@link HoodieKey}s from the Hoodie table, at the supplied instantTime {@link HoodieKey}s will be @@ -458,7 +459,7 @@ protected void preWrite(String instantTime, WriteOperationType writeOperationTyp * @param hoodieTable Hoodie Table * @return Write Status */ - protected abstract O postWrite(HoodieWriteMetadata result, String instantTime, HoodieTable hoodieTable); + protected abstract O postWrite(HoodieWriteMetadata result, String instantTime, HoodieTable hoodieTable); /** * Post Commit Hook. Derived classes use this method to perform post-commit processing @@ -468,7 +469,7 @@ protected void preWrite(String instantTime, WriteOperationType writeOperationTyp * @param instantTime Instant Time * @param extraMetadata Additional Metadata passed by user */ - protected void postCommit(HoodieTable table, HoodieCommitMetadata metadata, String instantTime, Option> extraMetadata) { + protected void postCommit(HoodieTable table, HoodieCommitMetadata metadata, String instantTime, Option> extraMetadata) { try { // Delete the marker directory for the instant. WriteMarkersFactory.get(config.getMarkersType(), table, instantTime) @@ -480,7 +481,7 @@ protected void postCommit(HoodieTable table, HoodieCommitMetadata me } } - protected void runTableServicesInline(HoodieTable table, HoodieCommitMetadata metadata, Option> extraMetadata) { + protected void runTableServicesInline(HoodieTable table, HoodieCommitMetadata metadata, Option> extraMetadata) { if (!tableServicesEnabled(config)) { return; } @@ -524,7 +525,7 @@ protected void runTableServicesInline(HoodieTable table, HoodieCommi } } - protected void runAnyPendingCompactions(HoodieTable table) { + protected void runAnyPendingCompactions(HoodieTable table) { table.getActiveTimeline().getWriteTimeline().filterPendingCompactionTimeline().getInstants() .forEach(instant -> { LOG.info("Running previously failed inflight compaction at instant " + instant); @@ -532,7 +533,7 @@ protected void runAnyPendingCompactions(HoodieTable table) { }); } - protected void runAnyPendingClustering(HoodieTable table) { + protected void runAnyPendingClustering(HoodieTable table) { table.getActiveTimeline().filterPendingReplaceTimeline().getInstants().forEach(instant -> { Option> instantPlan = ClusteringUtils.getClusteringPlan(table.getMetaClient(), instant); if (instantPlan.isPresent()) { @@ -558,7 +559,7 @@ protected void autoCleanOnCommit() { } } - protected void autoArchiveOnCommit(HoodieTable table) { + protected void autoArchiveOnCommit(HoodieTable table) { if (!config.isAutoArchive()) { return; } @@ -808,7 +809,7 @@ public HoodieCleanMetadata clean(boolean skipLocking) { * and keep increasing unbounded over time. * @param table table to commit on. */ - protected void archive(HoodieTable table) { + protected void archive(HoodieTable table) { if (!tableServicesEnabled(config)) { return; } @@ -937,7 +938,7 @@ public abstract void commitCompaction(String compactionInstantTime, HoodieCommit /** * Commit Compaction and track metrics. */ - protected abstract void completeCompaction(HoodieCommitMetadata metadata, HoodieTable table, String compactionCommitTime); + protected abstract void completeCompaction(HoodieCommitMetadata metadata, HoodieTable table, String compactionCommitTime); /** * Get inflight time line exclude compaction and clustering. @@ -1223,7 +1224,7 @@ protected Option inlineScheduleClustering(Option> ex return scheduleClustering(extraMetadata); } - protected void rollbackInflightClustering(HoodieInstant inflightInstant, HoodieTable table) { + protected void rollbackInflightClustering(HoodieInstant inflightInstant, HoodieTable table) { Option pendingRollbackInstantInfo = getPendingRollbackInfo(table.getMetaClient(), inflightInstant.getTimestamp(), false); String commitTime = pendingRollbackInstantInfo.map(entry -> entry.getRollbackInstant().getTimestamp()).orElse(HoodieActiveTimeline.createNewInstantTime()); table.scheduleRollback(context, commitTime, inflightInstant, false, config.shouldRollbackUsingMarkers()); @@ -1238,7 +1239,7 @@ protected void rollbackInflightClustering(HoodieInstant inflightInstant, HoodieT * @param instantTime Instant Time * @param stats Hoodie Write Stat */ - protected void finalizeWrite(HoodieTable table, String instantTime, List stats) { + protected void finalizeWrite(HoodieTable table, String instantTime, List stats) { try { final Timer.Context finalizeCtx = metrics.getFinalizeCtx(); table.finalizeWrite(context, instantTime, stats); @@ -1273,7 +1274,7 @@ public HoodieMetrics getMetrics() { * @param instantTime current inflight instant time * @return instantiated {@link HoodieTable} */ - protected abstract HoodieTable doInitTable(HoodieTableMetaClient metaClient, Option instantTime); + protected abstract HoodieTable doInitTable(HoodieTableMetaClient metaClient, Option instantTime); /** * Instantiates and initializes instance of {@link HoodieTable}, performing crucial bootstrapping @@ -1288,14 +1289,14 @@ public HoodieMetrics getMetrics() { *
  • Initializing metrics contexts
  • * */ - protected final HoodieTable initTable(WriteOperationType operationType, Option instantTime) { + protected final HoodieTable initTable(WriteOperationType operationType, Option instantTime) { HoodieTableMetaClient metaClient = createMetaClient(true); // Setup write schemas for deletes if (operationType == WriteOperationType.DELETE) { setWriteSchemaForDeletes(metaClient); } - HoodieTable table; + HoodieTable table; this.txnManager.beginTransaction(); try { @@ -1381,7 +1382,7 @@ public void close() { this.txnManager.close(); } - private void setWriteTimer(HoodieTable table) { + private void setWriteTimer(HoodieTable table) { String commitType = table.getMetaClient().getCommitActionType(); if (commitType.equals(HoodieTimeline.COMMIT_ACTION)) { writeTimer = metrics.getCommitCtx(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java index e5262ad6bb9f2..62a4f089a45b5 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java @@ -180,7 +180,7 @@ public abstract HoodieWriteMetadata insert(HoodieEngineContext context, Strin * @return HoodieWriteMetadata */ public abstract HoodieWriteMetadata bulkInsert(HoodieEngineContext context, String instantTime, - I records, Option> bulkInsertPartitioner); + I records, Option bulkInsertPartitioner); /** * Deletes a list of {@link HoodieKey}s from the Hoodie table, at the supplied instantTime {@link HoodieKey}s will be @@ -237,7 +237,7 @@ public abstract HoodieWriteMetadata insertPrepped(HoodieEngineContext context * @return HoodieWriteMetadata */ public abstract HoodieWriteMetadata bulkInsertPrepped(HoodieEngineContext context, String instantTime, - I preppedRecords, Option> bulkInsertPartitioner); + I preppedRecords, Option bulkInsertPartitioner); /** * Replaces all the existing records and inserts the specified new records into Hoodie table at the supplied instantTime, diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/UpdateStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/UpdateStrategy.java index 009790812a624..4e33eb06038cd 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/UpdateStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/strategy/UpdateStrategy.java @@ -28,7 +28,7 @@ /** * When file groups in clustering, write records to these file group need to check. */ -public abstract class UpdateStrategy, I> { +public abstract class UpdateStrategy { protected final HoodieEngineContext engineContext; protected Set fileGroupsInPendingClustering; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseBulkInsertHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseBulkInsertHelper.java index dffd926aee3d5..ad2145c3501bf 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseBulkInsertHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseBulkInsertHelper.java @@ -34,7 +34,7 @@ public abstract class BaseBulkInsertHelper bulkInsert(I inputRecords, String instantTime, HoodieTable table, HoodieWriteConfig config, BaseCommitActionExecutor executor, boolean performDedupe, - Option> userDefinedBulkInsertPartitioner); + Option userDefinedBulkInsertPartitioner); /** * Only write input records. Does not change timeline/index. Return information about new files created. @@ -42,7 +42,7 @@ public abstract HoodieWriteMetadata bulkInsert(I inputRecords, String instant public abstract O bulkInsert(I inputRecords, String instantTime, HoodieTable table, HoodieWriteConfig config, boolean performDedupe, - Option> userDefinedBulkInsertPartitioner, + Option userDefinedBulkInsertPartitioner, boolean addMetadataFields, int parallelism, WriteHandleFactory writeHandleFactory); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieDeleteHelper.java similarity index 53% rename from hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteHelper.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieDeleteHelper.java index 381c115533897..fff52eb24d736 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieDeleteHelper.java @@ -7,19 +7,20 @@ * "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 + * 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. + * 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.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.EmptyHoodieRecordPayload; import org.apache.hudi.common.model.HoodieAvroRecord; @@ -28,16 +29,12 @@ 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.data.HoodieJavaRDD; 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; @@ -48,69 +45,64 @@ * @param */ @SuppressWarnings("checkstyle:LineLength") -public class SparkDeleteHelper extends - BaseDeleteHelper>, JavaRDD, JavaRDD, R> { - private SparkDeleteHelper() { +public class HoodieDeleteHelper extends + BaseDeleteHelper>, HoodieData, HoodieData, R> { + private HoodieDeleteHelper() { } private static class DeleteHelperHolder { - private static final SparkDeleteHelper SPARK_DELETE_HELPER = new SparkDeleteHelper(); + private static final HoodieDeleteHelper HOODIE_DELETE_HELPER = new HoodieDeleteHelper<>(); } - public static SparkDeleteHelper newInstance() { - return DeleteHelperHolder.SPARK_DELETE_HELPER; + public static HoodieDeleteHelper newInstance() { + return DeleteHelperHolder.HOODIE_DELETE_HELPER; } @Override - public JavaRDD deduplicateKeys(JavaRDD keys, HoodieTable>, JavaRDD, JavaRDD> table, int parallelism) { + public HoodieData deduplicateKeys(HoodieData keys, HoodieTable>, HoodieData, HoodieData> table, int parallelism) { boolean isIndexingGlobal = table.getIndex().isGlobal(); if (isIndexingGlobal) { - return keys.keyBy(HoodieKey::getRecordKey) - .reduceByKey((key1, key2) -> key1, parallelism) - .values(); + return keys.distinctWithKey(HoodieKey::getRecordKey, parallelism); } else { return keys.distinct(parallelism); } } @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); - + public HoodieWriteMetadata> execute(String instantTime, + HoodieData keys, + HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable>, HoodieData, HoodieData> table, + BaseCommitActionExecutor>, HoodieData, HoodieData, R> deleteExecutor) { try { - HoodieWriteMetadata result = null; - JavaRDD dedupedKeys = keys; + HoodieData dedupedKeys = keys; final int parallelism = config.getDeleteShuffleParallelism(); if (config.shouldCombineBeforeDelete()) { // De-dupe/merge if needed dedupedKeys = deduplicateKeys(keys, table, parallelism); - } else if (!keys.partitions().isEmpty()) { + } else if (!keys.isEmpty()) { dedupedKeys = keys.repartition(parallelism); } - JavaRDD> dedupedRecords = + HoodieData> dedupedRecords = dedupedKeys.map(key -> new HoodieAvroRecord(key, new EmptyHoodieRecordPayload())); Instant beginTag = Instant.now(); // perform index loop up to get existing location of records - JavaRDD> taggedRecords = HoodieJavaRDD.getJavaRDD( - table.getIndex().tagLocation(HoodieJavaRDD.of(dedupedRecords), context, table)); + HoodieData> taggedRecords = table.getIndex().tagLocation(dedupedRecords, context, table); Duration tagLocationDuration = Duration.between(beginTag, Instant.now()); // filter out non existent keys/records - JavaRDD> taggedValidRecords = taggedRecords.filter(HoodieRecord::isCurrentLocationKnown); + HoodieData> taggedValidRecords = taggedRecords.filter(HoodieRecord::isCurrentLocationKnown); + HoodieWriteMetadata> result; if (!taggedValidRecords.isEmpty()) { result = deleteExecutor.execute(taggedValidRecords); result.setIndexLookupDuration(tagLocationDuration); } else { // if entire set of keys are non existent deleteExecutor.saveWorkloadProfileMetadataToInflight(new WorkloadProfile(Pair.of(new HashMap<>(), new WorkloadStat())), instantTime); - result = new HoodieWriteMetadata(); - result.setWriteStatuses(jsc.emptyRDD()); + result = new HoodieWriteMetadata<>(); + result.setWriteStatuses(context.emptyHoodieData()); deleteExecutor.commitOnAutoCommit(result); } return result; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkMergeHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java similarity index 81% rename from hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkMergeHelper.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java index e87c3ef5ba77e..2b4a5d1608eec 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkMergeHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java @@ -19,6 +19,7 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -37,31 +38,29 @@ 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 BaseMergeHelper>, - JavaRDD, JavaRDD> { +public class HoodieMergeHelper extends + BaseMergeHelper>, HoodieData, HoodieData> { - private SparkMergeHelper() { + private HoodieMergeHelper() { } private static class MergeHelperHolder { - private static final SparkMergeHelper SPARK_MERGE_HELPER = new SparkMergeHelper(); + private static final HoodieMergeHelper HOODIE_MERGE_HELPER = new HoodieMergeHelper<>(); } - public static SparkMergeHelper newInstance() { - return SparkMergeHelper.MergeHelperHolder.SPARK_MERGE_HELPER; + public static HoodieMergeHelper newInstance() { + return MergeHelperHolder.HOODIE_MERGE_HELPER; } @Override - public void runMerge(HoodieTable>, JavaRDD, JavaRDD> table, - HoodieMergeHandle>, JavaRDD, JavaRDD> upsertHandle) throws IOException { + public void runMerge(HoodieTable>, HoodieData, HoodieData> table, + HoodieMergeHandle>, HoodieData, HoodieData> mergeHandle) 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; @@ -78,7 +77,7 @@ public void runMerge(HoodieTable>, JavaRDD } BoundedInMemoryExecutor wrapper = null; - HoodieFileReader reader = HoodieFileReaderFactory.getFileReader(cfgForHoodieFile, mergeHandle.getOldFilePath()); + HoodieFileReader reader = HoodieFileReaderFactory.getFileReader(cfgForHoodieFile, mergeHandle.getOldFilePath()); try { final Iterator readerIterator; if (baseFile.getBootstrapBaseFile().isPresent()) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkWriteHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieWriteHelper.java similarity index 60% rename from hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkWriteHelper.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieWriteHelper.java index 23dceb1382f34..b56d39b8e3679 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkWriteHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieWriteHelper.java @@ -19,60 +19,52 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieAvroRecord; 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.data.HoodieJavaRDD; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.table.HoodieTable; -import org.apache.spark.api.java.JavaRDD; +public class HoodieWriteHelper extends BaseWriteHelper>, + HoodieData, HoodieData, R> { -import scala.Tuple2; - -/** - * A spark implementation of {@link BaseWriteHelper}. - * - * @param - */ -public class SparkWriteHelper extends BaseWriteHelper>, - JavaRDD, JavaRDD, R> { - private SparkWriteHelper() { + private HoodieWriteHelper() { } private static class WriteHelperHolder { - private static final SparkWriteHelper SPARK_WRITE_HELPER = new SparkWriteHelper(); + private static final HoodieWriteHelper HOODIE_WRITE_HELPER = new HoodieWriteHelper<>(); } - public static SparkWriteHelper newInstance() { - return WriteHelperHolder.SPARK_WRITE_HELPER; + public static HoodieWriteHelper newInstance() { + return WriteHelperHolder.HOODIE_WRITE_HELPER; } @Override - protected JavaRDD> tag(JavaRDD> dedupedRecords, HoodieEngineContext context, - HoodieTable>, JavaRDD, JavaRDD> table) { - return HoodieJavaRDD.getJavaRDD( - table.getIndex().tagLocation(HoodieJavaRDD.of(dedupedRecords), context, table)); + protected HoodieData> tag(HoodieData> dedupedRecords, HoodieEngineContext context, + HoodieTable>, HoodieData, HoodieData> table) { + return table.getIndex().tagLocation(dedupedRecords, context, table); } @Override - public JavaRDD> deduplicateRecords( - JavaRDD> records, HoodieIndex index, int parallelism) { + public HoodieData> deduplicateRecords( + HoodieData> 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); + return Pair.of(key, record); }).reduceByKey((rec1, rec2) -> { @SuppressWarnings("unchecked") T reducedData = (T) rec2.getData().preCombine(rec1.getData()); HoodieKey reducedKey = rec1.getData().equals(reducedData) ? rec1.getKey() : rec2.getKey(); - return new HoodieAvroRecord(reducedKey, reducedData); - }, parallelism).map(Tuple2::_2); + return new HoodieAvroRecord<>(reducedKey, reducedData); + }, parallelism).map(Pair::getRight); } } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java index fb613309d3fa2..4523705f32b74 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkWriteClient.java @@ -18,8 +18,6 @@ package org.apache.hudi.client; -import com.codahale.metrics.Timer; -import org.apache.hadoop.conf.Configuration; import org.apache.hudi.async.AsyncCleanerService; import org.apache.hudi.client.common.HoodieFlinkEngineContext; import org.apache.hudi.common.data.HoodieList; @@ -55,6 +53,7 @@ import org.apache.hudi.io.MiniBatchHandle; import org.apache.hudi.metadata.FlinkHoodieBackedTableMetadataWriter; import org.apache.hudi.metadata.HoodieBackedTableMetadataWriter; +import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.table.HoodieFlinkTable; import org.apache.hudi.table.HoodieTable; @@ -64,6 +63,9 @@ import org.apache.hudi.table.upgrade.FlinkUpgradeDowngradeHelper; import org.apache.hudi.table.upgrade.UpgradeDowngrade; import org.apache.hudi.util.FlinkClientUtil; + +import com.codahale.metrics.Timer; +import org.apache.hadoop.conf.Configuration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -111,8 +113,8 @@ public boolean commit(String instantTime, List writeStatuses, Optio } @Override - protected HoodieTable>, List, List> createTable(HoodieWriteConfig config, Configuration hadoopConf, - boolean refreshTimeline) { + protected HoodieTable createTable(HoodieWriteConfig config, Configuration hadoopConf, + boolean refreshTimeline) { return HoodieFlinkTable.create(config, (HoodieFlinkEngineContext) context); } @@ -226,12 +228,12 @@ public List bulkInsert(List> records, String instan } @Override - public List bulkInsert(List> records, String instantTime, Option>>> userDefinedBulkInsertPartitioner) { + public List bulkInsert(List> records, String instantTime, Option userDefinedBulkInsertPartitioner) { throw new HoodieNotSupportedException("BulkInsert operation is not supported yet"); } @Override - public List bulkInsertPreppedRecords(List> preppedRecords, String instantTime, Option>>> bulkInsertPartitioner) { + public List bulkInsertPreppedRecords(List> preppedRecords, String instantTime, Option bulkInsertPartitioner) { throw new HoodieNotSupportedException("BulkInsertPrepped operation is not supported yet"); } @@ -304,7 +306,7 @@ public void waitForCleaningFinish() { @Override protected List postWrite(HoodieWriteMetadata> result, String instantTime, - HoodieTable>, List, List> hoodieTable) { + HoodieTable hoodieTable) { if (result.getIndexLookupDuration().isPresent()) { metrics.updateIndexMetrics(getOperationType().name(), result.getIndexUpdateDuration().get().toMillis()); } @@ -324,7 +326,7 @@ protected List postWrite(HoodieWriteMetadata> res * @param extraMetadata Additional Metadata passed by user */ @Override - protected void postCommit(HoodieTable>, List, List> table, + protected void postCommit(HoodieTable table, HoodieCommitMetadata metadata, String instantTime, Option> extraMetadata) { @@ -351,7 +353,7 @@ public void commitCompaction( @Override public void completeCompaction( HoodieCommitMetadata metadata, - HoodieTable>, List, List> table, + HoodieTable table, String compactionCommitTime) { this.context.setJobStatus(this.getClass().getSimpleName(), "Collect compaction write status and commit compaction"); List writeStats = metadata.getWriteStats(); @@ -363,7 +365,7 @@ public void completeCompaction( // Do not do any conflict resolution here as we do with regular writes. We take the lock here to ensure all writes to metadata table happens within a // single lock (single writer). Because more than one write to metadata table will result in conflicts since all of them updates the same partition. table.getMetadataWriter(compactionInstant.getTimestamp()).ifPresent( - w -> w.update(metadata, compactionInstant.getTimestamp(), table.isTableServiceAction(compactionInstant.getAction()))); + w -> ((HoodieTableMetadataWriter) w).update(metadata, compactionInstant.getTimestamp(), table.isTableServiceAction(compactionInstant.getAction()))); LOG.info("Committing Compaction {} finished with result {}.", compactionCommitTime, metadata); CompactHelpers.getInstance().completeInflightCompaction(table, compactionCommitTime, metadata); } finally { @@ -396,7 +398,7 @@ public HoodieWriteMetadata> cluster(final String clusteringIns } @Override - protected HoodieTable>, List, List> doInitTable(HoodieTableMetaClient metaClient, Option instantTime) { + protected HoodieTable doInitTable(HoodieTableMetaClient metaClient, Option instantTime) { // Create a Hoodie table which encapsulated the commits and files visible return getHoodieTable(); } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java index 7e41ab150fbf2..14937d6fee244 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java @@ -68,6 +68,7 @@ import org.slf4j.LoggerFactory; import javax.annotation.Nonnull; + import java.io.IOException; import java.util.Collections; import java.util.Iterator; @@ -231,7 +232,7 @@ public HoodieWriteMetadata> insert(HoodieEngineContext context public HoodieWriteMetadata> bulkInsert(HoodieEngineContext context, String instantTime, List> records, - Option>>> bulkInsertPartitioner) { + Option bulkInsertPartitioner) { throw new HoodieNotSupportedException("BulkInsert is not supported yet"); } @@ -264,7 +265,7 @@ public HoodieWriteMetadata> insertPrepped(HoodieEngineContext public HoodieWriteMetadata> bulkInsertPrepped(HoodieEngineContext context, String instantTime, List> preppedRecords, - Option>>> bulkInsertPartitioner) { + Option bulkInsertPartitioner) { throw new HoodieNotSupportedException("BulkInsertPrepped is not supported yet"); } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java index 9de9298c25ae9..faf46e08d5515 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java @@ -18,8 +18,6 @@ package org.apache.hudi.client; -import com.codahale.metrics.Timer; -import org.apache.hadoop.conf.Configuration; import org.apache.hudi.client.common.HoodieJavaEngineContext; import org.apache.hudi.client.embedded.EmbeddedTimelineService; import org.apache.hudi.common.data.HoodieList; @@ -43,6 +41,9 @@ import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.upgrade.JavaUpgradeDowngradeHelper; +import com.codahale.metrics.Timer; +import org.apache.hadoop.conf.Configuration; + import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -88,9 +89,9 @@ public boolean commit(String instantTime, } @Override - protected HoodieTable>, List, List> createTable(HoodieWriteConfig config, - Configuration hadoopConf, - boolean refreshTimeline) { + protected HoodieTable createTable(HoodieWriteConfig config, + Configuration hadoopConf, + boolean refreshTimeline) { return HoodieJavaTable.create(config, context); } @@ -152,7 +153,7 @@ public List bulkInsert(List> records, @Override public List bulkInsert(List> records, String instantTime, - Option>>> userDefinedBulkInsertPartitioner) { + Option userDefinedBulkInsertPartitioner) { throw new HoodieNotSupportedException("BulkInsert is not supported in HoodieJavaClient"); } @@ -166,7 +167,7 @@ public void transitionInflight(String instantTime) { @Override public List bulkInsertPreppedRecords(List> preppedRecords, String instantTime, - Option>>> bulkInsertPartitioner) { + Option bulkInsertPartitioner) { HoodieTable>, List, List> table = initTable(WriteOperationType.BULK_INSERT_PREPPED, Option.ofNullable(instantTime)); table.validateInsertSchema(); @@ -188,7 +189,7 @@ public List delete(List keys, @Override protected List postWrite(HoodieWriteMetadata> result, String instantTime, - HoodieTable>, List, List> hoodieTable) { + HoodieTable hoodieTable) { if (result.getIndexLookupDuration().isPresent()) { metrics.updateIndexMetrics(getOperationType().name(), result.getIndexUpdateDuration().get().toMillis()); } @@ -215,7 +216,7 @@ public void commitCompaction(String compactionInstantTime, @Override protected void completeCompaction(HoodieCommitMetadata metadata, - HoodieTable>, List, List> table, + HoodieTable table, String compactionCommitTime) { throw new HoodieNotSupportedException("CompleteCompaction is not supported in HoodieJavaClient"); } @@ -232,7 +233,7 @@ public HoodieWriteMetadata> cluster(final String clusteringIns } @Override - protected HoodieTable>, List, List> doInitTable(HoodieTableMetaClient metaClient, Option instantTime) { + protected HoodieTable doInitTable(HoodieTableMetaClient metaClient, Option instantTime) { // new JavaUpgradeDowngrade(metaClient, config, context).run(metaClient, HoodieTableVersion.current(), config, context, instantTime); // Create a Hoodie table which encapsulated the commits and files visible diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java index 447ed3e96cd9e..06c23049d974d 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java @@ -114,7 +114,7 @@ public HoodieWriteMetadata> insert(HoodieEngineContext context public HoodieWriteMetadata> bulkInsert(HoodieEngineContext context, String instantTime, List> records, - Option>>> bulkInsertPartitioner) { + Option bulkInsertPartitioner) { return new JavaBulkInsertCommitActionExecutor((HoodieJavaEngineContext) context, config, this, instantTime, records, bulkInsertPartitioner).execute(); } @@ -152,7 +152,7 @@ public HoodieWriteMetadata> insertPrepped(HoodieEngineContext public HoodieWriteMetadata> bulkInsertPrepped(HoodieEngineContext context, String instantTime, List> preppedRecords, - Option>>> bulkInsertPartitioner) { + Option bulkInsertPartitioner) { return new JavaBulkInsertPreppedCommitActionExecutor((HoodieJavaEngineContext) context, config, this, instantTime, preppedRecords, bulkInsertPartitioner).execute(); } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaMergeOnReadTable.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaMergeOnReadTable.java index 136c25b8cdb2a..32d30f704ecbb 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaMergeOnReadTable.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaMergeOnReadTable.java @@ -61,7 +61,7 @@ public HoodieWriteMetadata> upsertPrepped(HoodieEngineContext public HoodieWriteMetadata> bulkInsertPrepped(HoodieEngineContext context, String instantTime, List> preppedRecords, - Option>>> bulkInsertPartitioner) { + Option bulkInsertPartitioner) { return new JavaBulkInsertPreppedCommitActionExecutor((HoodieJavaEngineContext) context, config, this, instantTime, preppedRecords, bulkInsertPartitioner).execute(); } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertCommitActionExecutor.java index 9780262fb2b92..d5c7a0b0b5dc7 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertCommitActionExecutor.java @@ -36,17 +36,17 @@ public class JavaBulkInsertCommitActionExecutor> extends BaseJavaCommitActionExecutor { private final List> inputRecords; - private final Option>>> bulkInsertPartitioner; + private final Option bulkInsertPartitioner; public JavaBulkInsertCommitActionExecutor(HoodieJavaEngineContext context, HoodieWriteConfig config, HoodieTable table, String instantTime, List> inputRecords, - Option>>> bulkInsertPartitioner) { + Option bulkInsertPartitioner) { this(context, config, table, instantTime, inputRecords, bulkInsertPartitioner, Option.empty()); } public JavaBulkInsertCommitActionExecutor(HoodieJavaEngineContext context, HoodieWriteConfig config, HoodieTable table, String instantTime, List> inputRecords, - Option>>> bulkInsertPartitioner, + Option bulkInsertPartitioner, Option> extraMetadata) { super(context, config, table, instantTime, WriteOperationType.BULK_INSERT, extraMetadata); this.inputRecords = inputRecords; diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertHelper.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertHelper.java index de7afdf00ebeb..30f1d931a5462 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertHelper.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertHelper.java @@ -65,7 +65,7 @@ public HoodieWriteMetadata> bulkInsert(final List>, List, List, R> executor, final boolean performDedupe, - final Option>>> userDefinedBulkInsertPartitioner) { + final Option userDefinedBulkInsertPartitioner) { HoodieWriteMetadata result = new HoodieWriteMetadata(); // It's possible the transition to inflight could have already happened. @@ -89,7 +89,7 @@ public List bulkInsert(List> inputRecords, HoodieTable>, List, List> table, HoodieWriteConfig config, boolean performDedupe, - Option>>> userDefinedBulkInsertPartitioner, + Option userDefinedBulkInsertPartitioner, boolean useWriterSchema, int parallelism, WriteHandleFactory writeHandleFactory) { @@ -106,6 +106,7 @@ public List bulkInsert(List> inputRecords, BulkInsertPartitioner partitioner = userDefinedBulkInsertPartitioner.isPresent() ? userDefinedBulkInsertPartitioner.get() : JavaBulkInsertInternalPartitionerFactory.get(config.getBulkInsertSortMode()); + // only List is supported for Java partitioner, but it is not enforced by BulkInsertPartitioner API. To improve this, TODO HUDI-3463 repartitionedRecords = (List>) partitioner.repartitionRecords(dedupedRecords, parallelism); FileIdPrefixProvider fileIdPrefixProvider = (FileIdPrefixProvider) ReflectionUtils.loadClass( diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertPreppedCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertPreppedCommitActionExecutor.java index ed72fbe7850e2..14c4c8a93e916 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertPreppedCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertPreppedCommitActionExecutor.java @@ -36,12 +36,12 @@ public class JavaBulkInsertPreppedCommitActionExecutor { private final List> preppedInputRecord; - private final Option>>> userDefinedBulkInsertPartitioner; + private final Option userDefinedBulkInsertPartitioner; public JavaBulkInsertPreppedCommitActionExecutor(HoodieJavaEngineContext context, HoodieWriteConfig config, HoodieTable table, String instantTime, List> preppedInputRecord, - Option>>> userDefinedBulkInsertPartitioner) { + Option userDefinedBulkInsertPartitioner) { super(context, config, table, instantTime, WriteOperationType.BULK_INSERT); this.preppedInputRecord = preppedInputRecord; this.userDefinedBulkInsertPartitioner = userDefinedBulkInsertPartitioner; diff --git a/hudi-client/hudi-spark-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 index e9bdc427e8356..37a78a4be54c0 100644 --- a/hudi-client/hudi-spark-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 @@ -68,7 +68,7 @@ public class HoodieReadClient> implements Seria * base path pointing to the table. Until, then just always assume a BloomIndex */ private final transient HoodieIndex index; - private HoodieTable>, JavaRDD, JavaRDD> hoodieTable; + private HoodieTable hoodieTable; private transient Option sqlContextOpt; private final transient HoodieSparkEngineContext context; private final transient Configuration hadoopConf; 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 index ec1ecd6a0c291..ac9259c51ad70 100644 --- 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 @@ -18,11 +18,10 @@ package org.apache.hudi.client; -import com.codahale.metrics.Timer; -import org.apache.hadoop.conf.Configuration; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.client.embedded.EmbeddedTimelineService; import org.apache.hudi.client.utils.TransactionUtils; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.metrics.Registry; @@ -45,6 +44,7 @@ import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.SparkHoodieIndexFactory; +import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.hudi.metrics.DistributedRegistry; import org.apache.hudi.table.BulkInsertPartitioner; @@ -54,6 +54,9 @@ import org.apache.hudi.table.action.compact.CompactHelpers; import org.apache.hudi.table.marker.WriteMarkersFactory; import org.apache.hudi.table.upgrade.SparkUpgradeDowngradeHelper; + +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; @@ -120,9 +123,9 @@ public boolean commit(String instantTime, JavaRDD writeStatuses, Op } @Override - protected HoodieTable>, JavaRDD, JavaRDD> createTable(HoodieWriteConfig config, - Configuration hadoopConf, - boolean refreshTimeline) { + protected HoodieTable createTable(HoodieWriteConfig config, + Configuration hadoopConf, + boolean refreshTimeline) { return HoodieSparkTable.create(config, context, refreshTimeline); } @@ -147,45 +150,49 @@ public void bootstrap(Option> extraMetadata) { @Override public JavaRDD upsert(JavaRDD> records, String instantTime) { - HoodieTable>, JavaRDD, JavaRDD> table = + HoodieTable>, HoodieData, HoodieData> table = initTable(WriteOperationType.UPSERT, Option.ofNullable(instantTime)); table.validateUpsertSchema(); preWrite(instantTime, WriteOperationType.UPSERT, table.getMetaClient()); - HoodieWriteMetadata> result = table.upsert(context, instantTime, records); + HoodieWriteMetadata> result = table.upsert(context, instantTime, HoodieJavaRDD.of(records)); + HoodieWriteMetadata> resultRDD = result.clone(HoodieJavaRDD.getJavaRDD(result.getWriteStatuses())); if (result.getIndexLookupDuration().isPresent()) { metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis()); } - return postWrite(result, instantTime, table); + return postWrite(resultRDD, instantTime, table); } @Override public JavaRDD upsertPreppedRecords(JavaRDD> preppedRecords, String instantTime) { - HoodieTable>, JavaRDD, JavaRDD> table = + HoodieTable>, HoodieData, HoodieData> table = initTable(WriteOperationType.UPSERT_PREPPED, Option.ofNullable(instantTime)); table.validateUpsertSchema(); preWrite(instantTime, WriteOperationType.UPSERT_PREPPED, table.getMetaClient()); - HoodieWriteMetadata> result = table.upsertPrepped(context,instantTime, preppedRecords); - return postWrite(result, instantTime, table); + HoodieWriteMetadata> result = table.upsertPrepped(context,instantTime, HoodieJavaRDD.of(preppedRecords)); + HoodieWriteMetadata> resultRDD = result.clone(HoodieJavaRDD.getJavaRDD(result.getWriteStatuses())); + return postWrite(resultRDD, instantTime, table); } @Override public JavaRDD insert(JavaRDD> records, String instantTime) { - HoodieTable>, JavaRDD, JavaRDD> table = + HoodieTable>, HoodieData, HoodieData> table = initTable(WriteOperationType.INSERT, Option.ofNullable(instantTime)); table.validateInsertSchema(); preWrite(instantTime, WriteOperationType.INSERT, table.getMetaClient()); - HoodieWriteMetadata> result = table.insert(context,instantTime, records); - return postWrite(result, instantTime, table); + HoodieWriteMetadata> result = table.insert(context,instantTime, HoodieJavaRDD.of(records)); + HoodieWriteMetadata> resultRDD = result.clone(HoodieJavaRDD.getJavaRDD(result.getWriteStatuses())); + return postWrite(resultRDD, instantTime, table); } @Override public JavaRDD insertPreppedRecords(JavaRDD> preppedRecords, String instantTime) { - HoodieTable>, JavaRDD, JavaRDD> table = + HoodieTable>, HoodieData, HoodieData> table = initTable(WriteOperationType.INSERT_PREPPED, Option.ofNullable(instantTime)); table.validateInsertSchema(); preWrite(instantTime, WriteOperationType.INSERT_PREPPED, table.getMetaClient()); - HoodieWriteMetadata> result = table.insertPrepped(context,instantTime, preppedRecords); - return postWrite(result, instantTime, table); + HoodieWriteMetadata> result = table.insertPrepped(context,instantTime, HoodieJavaRDD.of(preppedRecords)); + HoodieWriteMetadata> resultRDD = result.clone(HoodieJavaRDD.getJavaRDD(result.getWriteStatuses())); + return postWrite(resultRDD, instantTime, table); } /** @@ -196,11 +203,12 @@ public JavaRDD insertPreppedRecords(JavaRDD> preppe * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts */ public HoodieWriteResult insertOverwrite(JavaRDD> records, final String instantTime) { - HoodieTable table = initTable(WriteOperationType.INSERT_OVERWRITE, Option.ofNullable(instantTime)); + HoodieTable>, HoodieData, HoodieData> table = initTable(WriteOperationType.INSERT_OVERWRITE, Option.ofNullable(instantTime)); table.validateInsertSchema(); preWrite(instantTime, WriteOperationType.INSERT_OVERWRITE, table.getMetaClient()); - HoodieWriteMetadata result = table.insertOverwrite(context, instantTime, records); - return new HoodieWriteResult(postWrite(result, instantTime, table), result.getPartitionToReplaceFileIds()); + HoodieWriteMetadata> result = table.insertOverwrite(context, instantTime, HoodieJavaRDD.of(records)); + HoodieWriteMetadata> resultRDD = result.clone(HoodieJavaRDD.getJavaRDD(result.getWriteStatuses())); + return new HoodieWriteResult(postWrite(resultRDD, instantTime, table), result.getPartitionToReplaceFileIds()); } /** @@ -211,11 +219,12 @@ public HoodieWriteResult insertOverwrite(JavaRDD> records, final * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts */ public HoodieWriteResult insertOverwriteTable(JavaRDD> records, final String instantTime) { - HoodieTable table = initTable(WriteOperationType.INSERT_OVERWRITE_TABLE, Option.ofNullable(instantTime)); + HoodieTable>, HoodieData, HoodieData> table = initTable(WriteOperationType.INSERT_OVERWRITE_TABLE, Option.ofNullable(instantTime)); table.validateInsertSchema(); preWrite(instantTime, WriteOperationType.INSERT_OVERWRITE_TABLE, table.getMetaClient()); - HoodieWriteMetadata result = table.insertOverwriteTable(context, instantTime, records); - return new HoodieWriteResult(postWrite(result, instantTime, table), result.getPartitionToReplaceFileIds()); + HoodieWriteMetadata> result = table.insertOverwriteTable(context, instantTime, HoodieJavaRDD.of(records)); + HoodieWriteMetadata> resultRDD = result.clone(HoodieJavaRDD.getJavaRDD(result.getWriteStatuses())); + return new HoodieWriteResult(postWrite(resultRDD, instantTime, table), result.getPartitionToReplaceFileIds()); } @Override @@ -224,44 +233,48 @@ public JavaRDD bulkInsert(JavaRDD> records, String } @Override - public JavaRDD bulkInsert(JavaRDD> records, String instantTime, Option>>> userDefinedBulkInsertPartitioner) { - HoodieTable>, JavaRDD, JavaRDD> table = + public JavaRDD bulkInsert(JavaRDD> records, String instantTime, Option userDefinedBulkInsertPartitioner) { + HoodieTable>, HoodieData, HoodieData> table = initTable(WriteOperationType.BULK_INSERT, Option.ofNullable(instantTime)); table.validateInsertSchema(); preWrite(instantTime, WriteOperationType.BULK_INSERT, table.getMetaClient()); - HoodieWriteMetadata> result = table.bulkInsert(context,instantTime, records, userDefinedBulkInsertPartitioner); - return postWrite(result, instantTime, table); + HoodieWriteMetadata> result = table.bulkInsert(context,instantTime, HoodieJavaRDD.of(records), userDefinedBulkInsertPartitioner); + HoodieWriteMetadata> resultRDD = result.clone(HoodieJavaRDD.getJavaRDD(result.getWriteStatuses())); + return postWrite(resultRDD, instantTime, table); } @Override - public JavaRDD bulkInsertPreppedRecords(JavaRDD> preppedRecords, String instantTime, Option>>> bulkInsertPartitioner) { - HoodieTable>, JavaRDD, JavaRDD> table = + public JavaRDD bulkInsertPreppedRecords(JavaRDD> preppedRecords, String instantTime, Option bulkInsertPartitioner) { + HoodieTable>, HoodieData, HoodieData> table = initTable(WriteOperationType.BULK_INSERT_PREPPED, Option.ofNullable(instantTime)); table.validateInsertSchema(); preWrite(instantTime, WriteOperationType.BULK_INSERT_PREPPED, table.getMetaClient()); - HoodieWriteMetadata> result = table.bulkInsertPrepped(context,instantTime, preppedRecords, bulkInsertPartitioner); - return postWrite(result, instantTime, table); + HoodieWriteMetadata> result = table.bulkInsertPrepped(context,instantTime, HoodieJavaRDD.of(preppedRecords), bulkInsertPartitioner); + HoodieWriteMetadata> resultRDD = result.clone(HoodieJavaRDD.getJavaRDD(result.getWriteStatuses())); + return postWrite(resultRDD, instantTime, table); } @Override public JavaRDD delete(JavaRDD keys, String instantTime) { - HoodieTable>, JavaRDD, JavaRDD> table = initTable(WriteOperationType.DELETE, Option.ofNullable(instantTime)); + HoodieTable>, HoodieData, HoodieData> table = initTable(WriteOperationType.DELETE, Option.ofNullable(instantTime)); preWrite(instantTime, WriteOperationType.DELETE, table.getMetaClient()); - HoodieWriteMetadata> result = table.delete(context,instantTime, keys); - return postWrite(result, instantTime, table); + HoodieWriteMetadata> result = table.delete(context,instantTime, HoodieJavaRDD.of(keys)); + HoodieWriteMetadata> resultRDD = result.clone(HoodieJavaRDD.getJavaRDD(result.getWriteStatuses())); + return postWrite(resultRDD, instantTime, table); } public HoodieWriteResult deletePartitions(List partitions, String instantTime) { - HoodieTable>, JavaRDD, JavaRDD> table = initTable(WriteOperationType.DELETE_PARTITION, Option.ofNullable(instantTime)); + HoodieTable>, HoodieData, HoodieData> table = initTable(WriteOperationType.DELETE_PARTITION, Option.ofNullable(instantTime)); preWrite(instantTime, WriteOperationType.DELETE_PARTITION, table.getMetaClient()); - HoodieWriteMetadata> result = table.deletePartitions(context, instantTime, partitions); - return new HoodieWriteResult(postWrite(result, instantTime, table), result.getPartitionToReplaceFileIds()); + HoodieWriteMetadata> result = table.deletePartitions(context, instantTime, partitions); + HoodieWriteMetadata> resultRDD = result.clone(HoodieJavaRDD.getJavaRDD(result.getWriteStatuses())); + return new HoodieWriteResult(postWrite(resultRDD, instantTime, table), result.getPartitionToReplaceFileIds()); } @Override protected JavaRDD postWrite(HoodieWriteMetadata> result, String instantTime, - HoodieTable>, JavaRDD, JavaRDD> hoodieTable) { + HoodieTable hoodieTable) { if (result.getIndexLookupDuration().isPresent()) { metrics.updateIndexMetrics(getOperationType().name(), result.getIndexUpdateDuration().get().toMillis()); } @@ -288,7 +301,7 @@ public void commitCompaction(String compactionInstantTime, HoodieCommitMetadata @Override protected void completeCompaction(HoodieCommitMetadata metadata, - HoodieTable>, JavaRDD, JavaRDD> table, + HoodieTable table, String compactionCommitTime) { this.context.setJobStatus(this.getClass().getSimpleName(), "Collect compaction write status and commit compaction"); List writeStats = metadata.getWriteStats(); @@ -329,8 +342,8 @@ protected HoodieWriteMetadata> compact(String compactionIns table.getMetaClient().reloadActiveTimeline(); } compactionTimer = metrics.getCompactionCtx(); - HoodieWriteMetadata> compactionMetadata = - table.compact(context, compactionInstantTime); + HoodieWriteMetadata> writeMetadata = table.compact(context, compactionInstantTime); + HoodieWriteMetadata> compactionMetadata = writeMetadata.clone(HoodieJavaRDD.getJavaRDD(writeMetadata.getWriteStatuses())); if (shouldComplete && compactionMetadata.getCommitMetadata().isPresent()) { completeTableService(TableServiceType.COMPACT, compactionMetadata.getCommitMetadata().get(), table, compactionInstantTime); } @@ -349,7 +362,8 @@ public HoodieWriteMetadata> cluster(String clusteringInstan } clusteringTimer = metrics.getClusteringCtx(); LOG.info("Starting clustering at " + clusteringInstant); - HoodieWriteMetadata> clusteringMetadata = table.cluster(context, clusteringInstant); + HoodieWriteMetadata> writeMetadata = table.cluster(context, clusteringInstant); + HoodieWriteMetadata> clusteringMetadata = writeMetadata.clone(HoodieJavaRDD.getJavaRDD(writeMetadata.getWriteStatuses())); // TODO : Where is shouldComplete used ? if (shouldComplete && clusteringMetadata.getCommitMetadata().isPresent()) { completeTableService(TableServiceType.CLUSTER, clusteringMetadata.getCommitMetadata().get(), table, clusteringInstant); @@ -358,9 +372,8 @@ public HoodieWriteMetadata> cluster(String clusteringInstan } private void completeClustering(HoodieReplaceCommitMetadata metadata, - HoodieTable>, JavaRDD, JavaRDD> table, - String clusteringCommitTime) { - + HoodieTable table, + String clusteringCommitTime) { List writeStats = metadata.getPartitionToWriteStats().entrySet().stream().flatMap(e -> e.getValue().stream()).collect(Collectors.toList()); @@ -405,17 +418,17 @@ private void completeClustering(HoodieReplaceCommitMetadata metadata, LOG.info("Clustering successfully on commit " + clusteringCommitTime); } - private void updateTableMetadata(HoodieTable>, JavaRDD, JavaRDD> table, HoodieCommitMetadata commitMetadata, + private void updateTableMetadata(HoodieTable table, HoodieCommitMetadata commitMetadata, HoodieInstant hoodieInstant) { boolean isTableServiceAction = table.isTableServiceAction(hoodieInstant.getAction()); // Do not do any conflict resolution here as we do with regular writes. We take the lock here to ensure all writes to metadata table happens within a // single lock (single writer). Because more than one write to metadata table will result in conflicts since all of them updates the same partition. table.getMetadataWriter(hoodieInstant.getTimestamp()) - .ifPresent(writer -> writer.update(commitMetadata, hoodieInstant.getTimestamp(), isTableServiceAction)); + .ifPresent(writer -> ((HoodieTableMetadataWriter) writer).update(commitMetadata, hoodieInstant.getTimestamp(), isTableServiceAction)); } @Override - protected HoodieTable>, JavaRDD, JavaRDD> doInitTable(HoodieTableMetaClient metaClient, Option instantTime) { + protected HoodieTable doInitTable(HoodieTableMetaClient metaClient, Option instantTime) { // Initialize Metadata Table to make sure it's bootstrapped _before_ the operation, // if it didn't exist before // See https://issues.apache.org/jira/browse/HUDI-3343 for more details @@ -440,7 +453,7 @@ private void initializeMetadataTable(Option inFlightInstantTimestamp) { // TODO : To enforce priority between table service and ingestion writer, use transactions here and invoke strategy private void completeTableService(TableServiceType tableServiceType, HoodieCommitMetadata metadata, - HoodieTable>, JavaRDD, JavaRDD> table, + HoodieTable table, String commitInstant) { switch (tableServiceType) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java index 54c1c9f5ac05d..5a03cdf3bc9a1 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/MultipleSparkJobExecutionStrategy.java @@ -18,10 +18,6 @@ package org.apache.hudi.client.clustering.run.strategy; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieClusteringGroup; import org.apache.hudi.avro.model.HoodieClusteringPlan; @@ -62,6 +58,11 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.cluster.strategy.ClusteringExecutionStrategy; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +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.api.java.JavaRDD; @@ -94,7 +95,7 @@ public MultipleSparkJobExecutionStrategy(HoodieTable table, HoodieEngineContext public HoodieWriteMetadata> performClustering(final HoodieClusteringPlan clusteringPlan, final Schema schema, final String instantTime) { JavaSparkContext engineContext = HoodieSparkEngineContext.getSparkContext(getEngineContext()); // execute clustering for each group async and collect WriteStatus - Stream> writeStatusRDDStream = FutureUtils.allOf( + Stream> writeStatusesStream = FutureUtils.allOf( clusteringPlan.getInputGroups().stream() .map(inputGroup -> runClusteringForGroupAsync(inputGroup, clusteringPlan.getStrategy().getStrategyParams(), @@ -103,7 +104,7 @@ public HoodieWriteMetadata> performClustering(final Hood .collect(Collectors.toList())) .join() .stream(); - JavaRDD[] writeStatuses = convertStreamToArray(writeStatusRDDStream); + JavaRDD[] writeStatuses = convertStreamToArray(writeStatusesStream.map(HoodieJavaRDD::getJavaRDD)); JavaRDD writeStatusRDD = engineContext.union(writeStatuses); HoodieWriteMetadata> writeMetadata = new HoodieWriteMetadata<>(); @@ -125,7 +126,7 @@ public HoodieWriteMetadata> performClustering(final Hood * @param preserveHoodieMetadata Whether to preserve commit metadata while clustering. * @return RDD of {@link WriteStatus}. */ - public abstract JavaRDD performClusteringWithRecordsRDD(final JavaRDD> inputRecords, final int numOutputGroups, final String instantTime, + public abstract HoodieData performClusteringWithRecordsRDD(final HoodieData> inputRecords, final int numOutputGroups, final String instantTime, final Map strategyParams, final Schema schema, final List fileGroupIdList, final boolean preserveHoodieMetadata); @@ -164,11 +165,11 @@ protected Option>>> getPartitioner /** * Submit job to execute clustering for the group. */ - private CompletableFuture> runClusteringForGroupAsync(HoodieClusteringGroup clusteringGroup, Map strategyParams, + private CompletableFuture> runClusteringForGroupAsync(HoodieClusteringGroup clusteringGroup, Map strategyParams, boolean preserveHoodieMetadata, String instantTime) { return CompletableFuture.supplyAsync(() -> { JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(getEngineContext()); - JavaRDD> inputRecords = readRecordsForGroup(jsc, clusteringGroup, instantTime); + HoodieData> inputRecords = readRecordsForGroup(jsc, clusteringGroup, instantTime); Schema readerSchema = HoodieAvroUtils.addMetadataFields(new Schema.Parser().parse(getWriteConfig().getSchema())); List inputFileIds = clusteringGroup.getSlices().stream() .map(info -> new HoodieFileGroupId(info.getPartitionPath(), info.getFileId())) @@ -180,7 +181,7 @@ private CompletableFuture> runClusteringForGroupAsync(Hoodi /** * Get RDD of all records for the group. This includes all records from file slice (Apply updates from log files, if any). */ - private JavaRDD> readRecordsForGroup(JavaSparkContext jsc, HoodieClusteringGroup clusteringGroup, String instantTime) { + private HoodieData> readRecordsForGroup(JavaSparkContext jsc, HoodieClusteringGroup clusteringGroup, String instantTime) { List clusteringOps = clusteringGroup.getSlices().stream().map(ClusteringOperation::create).collect(Collectors.toList()); boolean hasLogFiles = clusteringOps.stream().anyMatch(op -> op.getDeltaFilePaths().size() > 0); if (hasLogFiles) { @@ -195,12 +196,12 @@ private JavaRDD> readRecordsForGroup(JavaSparkContext jsc, Hoodi /** * Read records from baseFiles, apply updates and convert to RDD. */ - private JavaRDD> readRecordsForGroupWithLogs(JavaSparkContext jsc, + private HoodieData> readRecordsForGroupWithLogs(JavaSparkContext jsc, List clusteringOps, String instantTime) { HoodieWriteConfig config = getWriteConfig(); HoodieTable table = getHoodieTable(); - return jsc.parallelize(clusteringOps, clusteringOps.size()).mapPartitions(clusteringOpsPartition -> { + return HoodieJavaRDD.of(jsc.parallelize(clusteringOps, clusteringOps.size()).mapPartitions(clusteringOpsPartition -> { List>> recordIterators = new ArrayList<>(); clusteringOpsPartition.forEachRemaining(clusteringOp -> { long maxMemoryPerCompaction = IOUtils.getMaxMemoryPerCompaction(new SparkTaskContextSupplier(), config); @@ -237,20 +238,20 @@ private JavaRDD> readRecordsForGroupWithLogs(JavaSparkContext js }); return new ConcatenatingIterator<>(recordIterators); - }); + })); } /** * Read records from baseFiles and convert to RDD. */ - private JavaRDD> readRecordsForGroupBaseFiles(JavaSparkContext jsc, + private HoodieData> readRecordsForGroupBaseFiles(JavaSparkContext jsc, List clusteringOps) { SerializableConfiguration hadoopConf = new SerializableConfiguration(getHoodieTable().getHadoopConf()); HoodieWriteConfig writeConfig = getWriteConfig(); // NOTE: It's crucial to make sure that we don't capture whole "this" object into the // closure, as this might lead to issues attempting to serialize its nested fields - return jsc.parallelize(clusteringOps, clusteringOps.size()) + return HoodieJavaRDD.of(jsc.parallelize(clusteringOps, clusteringOps.size()) .mapPartitions(clusteringOpsPartition -> { List> iteratorsForPartition = new ArrayList<>(); clusteringOpsPartition.forEachRemaining(clusteringOp -> { @@ -266,7 +267,7 @@ private JavaRDD> readRecordsForGroupBaseFiles(JavaSparkContext j return new ConcatenatingIterator<>(iteratorsForPartition); }) - .map(record -> transform(record, writeConfig)); + .map(record -> transform(record, writeConfig))); } /** diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java index 2a503931b496b..4dedabaec850e 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSingleFileSortExecutionStrategy.java @@ -20,6 +20,7 @@ package org.apache.hudi.client.clustering.run.strategy; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecord; @@ -34,7 +35,6 @@ import org.apache.avro.Schema; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaRDD; import java.util.List; import java.util.Map; @@ -56,7 +56,7 @@ public SparkSingleFileSortExecutionStrategy(HoodieTable table, } @Override - public JavaRDD performClusteringWithRecordsRDD(JavaRDD> inputRecords, + public HoodieData performClusteringWithRecordsRDD(HoodieData> inputRecords, int numOutputGroups, String instantTime, Map strategyParams, @@ -74,7 +74,7 @@ public JavaRDD performClusteringWithRecordsRDD(JavaRDD) SparkBulkInsertHelper.newInstance().bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig, + return (HoodieData) SparkBulkInsertHelper.newInstance().bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig, false, getPartitioner(strategyParams, schema), true, numOutputGroups, new SingleFileHandleCreateFactory(fileGroupIdList.get(0).getFileId(), preserveHoodieMetadata)); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java index 22d5300215d0b..d664c835e494b 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/run/strategy/SparkSortAndSizeExecutionStrategy.java @@ -19,6 +19,7 @@ package org.apache.hudi.client.clustering.run.strategy; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecord; @@ -32,7 +33,6 @@ import org.apache.avro.Schema; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaRDD; import java.util.List; import java.util.Map; @@ -54,7 +54,7 @@ public SparkSortAndSizeExecutionStrategy(HoodieTable table, } @Override - public JavaRDD performClusteringWithRecordsRDD(final JavaRDD> inputRecords, final int numOutputGroups, + public HoodieData performClusteringWithRecordsRDD(final HoodieData> inputRecords, final int numOutputGroups, final String instantTime, final Map strategyParams, final Schema schema, final List fileGroupIdList, final boolean preserveHoodieMetadata) { LOG.info("Starting clustering for a group, parallelism:" + numOutputGroups + " commit:" + instantTime); @@ -64,7 +64,7 @@ public JavaRDD performClusteringWithRecordsRDD(final JavaRDD) SparkBulkInsertHelper.newInstance() + return (HoodieData) SparkBulkInsertHelper.newInstance() .bulkInsert(inputRecords, instantTime, getHoodieTable(), newConfig, false, getPartitioner(strategyParams, schema), true, numOutputGroups, new CreateHandleFactory(preserveHoodieMetadata)); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/BaseSparkUpdateStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/BaseSparkUpdateStrategy.java index 655c11983e46b..3eadba25bbcfb 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/BaseSparkUpdateStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/BaseSparkUpdateStrategy.java @@ -19,13 +19,12 @@ package org.apache.hudi.client.clustering.update.strategy; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.table.action.cluster.strategy.UpdateStrategy; -import org.apache.spark.api.java.JavaRDD; - import java.util.List; import java.util.Set; @@ -33,7 +32,7 @@ * Spark base update strategy, write records to the file groups which are in clustering * need to check. Spark relate implementations should extend this base class. */ -public abstract class BaseSparkUpdateStrategy> extends UpdateStrategy>> { +public abstract class BaseSparkUpdateStrategy> extends UpdateStrategy>> { public BaseSparkUpdateStrategy(HoodieSparkEngineContext engineContext, Set fileGroupsInPendingClustering) { @@ -45,9 +44,9 @@ public BaseSparkUpdateStrategy(HoodieSparkEngineContext engineContext, * @param inputRecords the records to write, tagged with target file id * @return the records matched file group ids */ - protected List getGroupIdsWithUpdate(JavaRDD> inputRecords) { + protected List getGroupIdsWithUpdate(HoodieData> inputRecords) { return inputRecords .filter(record -> record.getCurrentLocation() != null) - .map(record -> new HoodieFileGroupId(record.getPartitionPath(), record.getCurrentLocation().getFileId())).distinct().collect(); + .map(record -> new HoodieFileGroupId(record.getPartitionPath(), record.getCurrentLocation().getFileId())).distinct().collectAsList(); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkAllowUpdateStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkAllowUpdateStrategy.java index 92a5fb69a7cd9..59040629f718e 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkAllowUpdateStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkAllowUpdateStrategy.java @@ -19,13 +19,12 @@ package org.apache.hudi.client.clustering.update.strategy; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieFileGroupId; 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.spark.api.java.JavaRDD; - import java.util.HashSet; import java.util.List; import java.util.Set; @@ -42,7 +41,7 @@ public SparkAllowUpdateStrategy(HoodieSparkEngineContext engineContext, } @Override - public Pair>, Set> handleUpdate(JavaRDD> taggedRecordsRDD) { + public Pair>, Set> handleUpdate(HoodieData> taggedRecordsRDD) { List fileGroupIdsWithRecordUpdate = getGroupIdsWithUpdate(taggedRecordsRDD); Set fileGroupIdsWithUpdatesAndPendingClustering = fileGroupIdsWithRecordUpdate.stream() .filter(f -> fileGroupsInPendingClustering.contains(f)) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkRejectUpdateStrategy.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkRejectUpdateStrategy.java index ac058a4d85095..d09422ee77242 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkRejectUpdateStrategy.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/clustering/update/strategy/SparkRejectUpdateStrategy.java @@ -19,6 +19,7 @@ package org.apache.hudi.client.clustering.update.strategy; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; @@ -27,7 +28,6 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaRDD; import java.util.Collections; import java.util.HashSet; @@ -47,7 +47,7 @@ public SparkRejectUpdateStrategy(HoodieSparkEngineContext engineContext, } @Override - public Pair>, Set> handleUpdate(JavaRDD> taggedRecordsRDD) { + public Pair>, Set> handleUpdate(HoodieData> taggedRecordsRDD) { List fileGroupIdsWithRecordUpdate = getGroupIdsWithUpdate(taggedRecordsRDD); fileGroupIdsWithRecordUpdate.forEach(fileGroupIdWithRecordUpdate -> { if (fileGroupsInPendingClustering.contains(fileGroupIdWithRecordUpdate)) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkValidatorUtils.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkValidatorUtils.java index 9e72390e49f55..fd083f2c89a46 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkValidatorUtils.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkValidatorUtils.java @@ -21,7 +21,9 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.client.validator.SparkPreCommitValidator; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.BaseFile; import org.apache.hudi.common.table.view.HoodieTablePreCommitFileSystemView; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.StringUtils; @@ -34,13 +36,11 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SQLContext; import java.util.Arrays; -import java.util.HashSet; import java.util.List; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -61,7 +61,7 @@ public class SparkValidatorUtils { * Throw error if there are validation failures. */ public static void runValidators(HoodieWriteConfig config, - HoodieWriteMetadata> writeMetadata, + HoodieWriteMetadata> writeMetadata, HoodieEngineContext context, HoodieTable table, String instantTime) { @@ -69,10 +69,10 @@ public static void runValidators(HoodieWriteConfig config, LOG.info("no validators configured."); } else { if (!writeMetadata.getWriteStats().isPresent()) { - writeMetadata.setWriteStats(writeMetadata.getWriteStatuses().map(WriteStatus::getStat).collect()); + writeMetadata.setWriteStats(writeMetadata.getWriteStatuses().map(WriteStatus::getStat).collectAsList()); } - Set partitionsModified = new HashSet<>(writeMetadata.getWriteStats().get().stream().map(writeStats -> - writeStats.getPartitionPath()).collect(Collectors.toList())); + Set partitionsModified = writeMetadata.getWriteStats().get().stream().map(writeStats -> + writeStats.getPartitionPath()).collect(Collectors.toSet()); SQLContext sqlContext = new SQLContext(HoodieSparkEngineContext.getSparkContext(context)); // Refresh timeline to ensure validator sees the any other operations done on timeline (async operations such as other clustering/compaction/rollback) table.getMetaClient().reloadActiveTimeline(); @@ -101,8 +101,8 @@ public static void runValidators(HoodieWriteConfig config, /** * Run validators in a separate thread pool for parallelism. Each of validator can submit a distributed spark job if needed. */ - private static CompletableFuture runValidatorAsync(SparkPreCommitValidator validator, HoodieWriteMetadata writeMetadata, - Dataset beforeState, Dataset afterState, String instantTime) { + private static CompletableFuture runValidatorAsync(SparkPreCommitValidator validator, HoodieWriteMetadata> writeMetadata, + Dataset beforeState, Dataset afterState, String instantTime) { return CompletableFuture.supplyAsync(() -> { try { validator.validate(instantTime, writeMetadata, beforeState, afterState); @@ -120,10 +120,10 @@ private static CompletableFuture runValidatorAsync(SparkPreCommitValida * Note that this only works for COW tables. */ public static Dataset getRecordsFromCommittedFiles(SQLContext sqlContext, - Set partitionsAffected, HoodieTable table) { + Set partitionsAffected, HoodieTable table) { List committedFiles = partitionsAffected.stream() - .flatMap(partition -> table.getBaseFileOnlyView().getLatestBaseFiles(partition).map(bf -> bf.getPath())) + .flatMap(partition -> table.getBaseFileOnlyView().getLatestBaseFiles(partition).map(BaseFile::getPath)) .collect(Collectors.toList()); if (committedFiles.isEmpty()) { @@ -145,7 +145,7 @@ public static Dataset readRecordsForBaseFiles(SQLContext sqlContext, List getRecordsFromPendingCommits(SQLContext sqlContext, Set partitionsAffected, - HoodieWriteMetadata> writeMetadata, + HoodieWriteMetadata> writeMetadata, HoodieTable table, String instantTime) { @@ -157,7 +157,7 @@ public static Dataset getRecordsFromPendingCommits(SQLContext sqlContext, instantTime); List newFiles = partitionsAffected.stream() - .flatMap(partition -> fsView.getLatestBaseFiles(partition).map(bf -> bf.getPath())) + .flatMap(partition -> fsView.getLatestBaseFiles(partition).map(BaseFile::getPath)) .collect(Collectors.toList()); if (newFiles.isEmpty()) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SparkPreCommitValidator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SparkPreCommitValidator.java index f12d337bb12e9..f08d11b571492 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SparkPreCommitValidator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SparkPreCommitValidator.java @@ -19,6 +19,7 @@ package org.apache.hudi.client.validator; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieWriteStat; @@ -28,9 +29,9 @@ import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; 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.sql.Dataset; import org.apache.spark.sql.Row; @@ -41,7 +42,7 @@ /** * Validator can be configured pre-commit. */ -public abstract class SparkPreCommitValidator> { +public abstract class SparkPreCommitValidator> { private static final Logger LOG = LogManager.getLogger(SparkPreCommitValidator.class); private HoodieSparkTable table; @@ -59,7 +60,7 @@ protected Set getPartitionsModified(HoodieWriteMetadata writeResult) if (writeResult.getWriteStats().isPresent()) { partitionsModified = writeResult.getWriteStats().get().stream().map(HoodieWriteStat::getPartitionPath).collect(Collectors.toSet()); } else { - partitionsModified = new HashSet<>(writeResult.getWriteStatuses().map(WriteStatus::getPartitionPath).collect()); + partitionsModified = new HashSet<>(writeResult.getWriteStatuses().map(WriteStatus::getPartitionPath).collectAsList()); } return partitionsModified; } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryEqualityPreCommitValidator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryEqualityPreCommitValidator.java index b27f84e41c28c..2506d52b4a416 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryEqualityPreCommitValidator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryEqualityPreCommitValidator.java @@ -19,6 +19,7 @@ package org.apache.hudi.client.validator; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodiePreCommitValidatorConfig; @@ -28,7 +29,6 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SQLContext; @@ -40,7 +40,7 @@ * * Expects both queries to return same result. */ -public class SqlQueryEqualityPreCommitValidator> extends SqlQueryPreCommitValidator { +public class SqlQueryEqualityPreCommitValidator> extends SqlQueryPreCommitValidator { private static final Logger LOG = LogManager.getLogger(SqlQueryEqualityPreCommitValidator.class); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryInequalityPreCommitValidator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryInequalityPreCommitValidator.java index 026334fde0cde..8a25150651943 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryInequalityPreCommitValidator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryInequalityPreCommitValidator.java @@ -19,6 +19,7 @@ package org.apache.hudi.client.validator; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodiePreCommitValidatorConfig; @@ -28,7 +29,6 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SQLContext; @@ -40,7 +40,7 @@ *

    * Expects query results do not match. */ -public class SqlQueryInequalityPreCommitValidator> extends SqlQueryPreCommitValidator { +public class SqlQueryInequalityPreCommitValidator> extends SqlQueryPreCommitValidator { private static final Logger LOG = LogManager.getLogger(SqlQueryInequalityPreCommitValidator.class); public SqlQueryInequalityPreCommitValidator(HoodieSparkTable table, HoodieEngineContext engineContext, HoodieWriteConfig config) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryPreCommitValidator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryPreCommitValidator.java index 122cf2be5c841..3a88d54d36d7b 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryPreCommitValidator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQueryPreCommitValidator.java @@ -20,15 +20,16 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieValidationException; import org.apache.hudi.table.HoodieSparkTable; + 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.Dataset; import org.apache.spark.sql.Row; @@ -41,7 +42,7 @@ /** * Validator framework to run sql queries and compare table state at different locations. */ -public abstract class SqlQueryPreCommitValidator> extends SparkPreCommitValidator { +public abstract class SqlQueryPreCommitValidator> extends SparkPreCommitValidator { private static final Logger LOG = LogManager.getLogger(SqlQueryPreCommitValidator.class); private static final AtomicInteger TABLE_COUNTER = new AtomicInteger(0); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQuerySingleResultPreCommitValidator.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQuerySingleResultPreCommitValidator.java index 66e956dc59650..b1942244d3d39 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQuerySingleResultPreCommitValidator.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/validator/SqlQuerySingleResultPreCommitValidator.java @@ -19,6 +19,7 @@ package org.apache.hudi.client.validator; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.config.HoodiePreCommitValidatorConfig; @@ -28,7 +29,6 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaRDD; import org.apache.spark.sql.Row; import org.apache.spark.sql.SQLContext; @@ -40,7 +40,7 @@ *

    * Example configuration: "query1#expectedResult1;query2#expectedResult2;" */ -public class SqlQuerySingleResultPreCommitValidator> extends SqlQueryPreCommitValidator { +public class SqlQuerySingleResultPreCommitValidator> extends SqlQueryPreCommitValidator { private static final Logger LOG = LogManager.getLogger(SqlQueryInequalityPreCommitValidator.class); public SqlQuerySingleResultPreCommitValidator(HoodieSparkTable table, HoodieEngineContext engineContext, HoodieWriteConfig config) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaPairRDD.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaPairRDD.java index ffa1a35652c3c..ddcaaec0fa6ca 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaPairRDD.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaPairRDD.java @@ -21,6 +21,7 @@ import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.data.HoodiePairData; +import org.apache.hudi.common.function.SerializableBiFunction; import org.apache.hudi.common.function.SerializableFunction; import org.apache.hudi.common.function.SerializablePairFunction; import org.apache.hudi.common.util.Option; @@ -103,6 +104,11 @@ public Map countByKey() { return pairRDDData.countByKey(); } + @Override + public HoodiePairData reduceByKey(SerializableBiFunction func, int parallelism) { + return HoodieJavaPairRDD.of(pairRDDData.reduceByKey(func::apply, parallelism)); + } + @Override public HoodieData map(SerializableFunction, O> func) { return HoodieJavaRDD.of(pairRDDData.map( diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaRDD.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaRDD.java index 1381ea86e481c..66edf607f84dd 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaRDD.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/data/HoodieJavaRDD.java @@ -130,6 +130,23 @@ public HoodieData distinct() { return HoodieJavaRDD.of(rddData.distinct()); } + @Override + public HoodieData distinct(int parallelism) { + return HoodieJavaRDD.of(rddData.distinct(parallelism)); + } + + @Override + public HoodieData distinctWithKey(SerializableFunction keyGetter, int parallelism) { + return mapToPair(i -> Pair.of(keyGetter.apply(i), i)) + .reduceByKey((value1, value2) -> value1, parallelism) + .values(); + } + + @Override + public HoodieData filter(SerializableFunction filterFunc) { + return HoodieJavaRDD.of(rddData.filter(filterFunc::apply)); + } + @Override public HoodieData union(HoodieData other) { return HoodieJavaRDD.of(rddData.union((JavaRDD) other.get())); @@ -139,4 +156,9 @@ public HoodieData union(HoodieData other) { public List collectAsList() { return rddData.collect(); } + + @Override + public HoodieData repartition(int parallelism) { + return HoodieJavaRDD.of(rddData.repartition(parallelism)); + } } 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 index 31bd436612c11..8f5211212253a 100644 --- 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 @@ -32,6 +32,7 @@ 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.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieKey; @@ -61,6 +62,7 @@ import org.apache.hudi.table.action.clean.CleanPlanActionExecutor; import org.apache.hudi.table.action.cluster.ClusteringPlanActionExecutor; import org.apache.hudi.table.action.cluster.SparkExecuteClusteringCommitActionExecutor; +import org.apache.hudi.table.action.commit.HoodieMergeHelper; import org.apache.hudi.table.action.commit.SparkBulkInsertCommitActionExecutor; import org.apache.hudi.table.action.commit.SparkBulkInsertPreppedCommitActionExecutor; import org.apache.hudi.table.action.commit.SparkDeleteCommitActionExecutor; @@ -69,7 +71,6 @@ import org.apache.hudi.table.action.commit.SparkInsertOverwriteCommitActionExecutor; import org.apache.hudi.table.action.commit.SparkInsertOverwriteTableCommitActionExecutor; 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.CopyOnWriteRestoreActionExecutor; @@ -82,7 +83,6 @@ import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.spark.api.java.JavaRDD; import javax.annotation.Nonnull; @@ -117,58 +117,58 @@ public boolean isTableServiceAction(String actionType) { } @Override - public HoodieWriteMetadata> upsert(HoodieEngineContext context, String instantTime, JavaRDD> records) { + public HoodieWriteMetadata> upsert(HoodieEngineContext context, String instantTime, HoodieData> records) { return new SparkUpsertCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, records).execute(); } @Override - public HoodieWriteMetadata> insert(HoodieEngineContext context, String instantTime, JavaRDD> records) { + public HoodieWriteMetadata> insert(HoodieEngineContext context, String instantTime, HoodieData> 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, + public HoodieWriteMetadata> bulkInsert(HoodieEngineContext context, String instantTime, HoodieData> records, + Option userDefinedBulkInsertPartitioner) { + return new SparkBulkInsertCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, records, userDefinedBulkInsertPartitioner).execute(); } @Override - public HoodieWriteMetadata> delete(HoodieEngineContext context, String instantTime, JavaRDD keys) { + public HoodieWriteMetadata> delete(HoodieEngineContext context, String instantTime, HoodieData keys) { return new SparkDeleteCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, keys).execute(); } @Override - public HoodieWriteMetadata deletePartitions(HoodieEngineContext context, String instantTime, List partitions) { - return new SparkDeletePartitionCommitActionExecutor(context, config, this, instantTime, partitions).execute(); + public HoodieWriteMetadata> deletePartitions(HoodieEngineContext context, String instantTime, List partitions) { + return new SparkDeletePartitionCommitActionExecutor<>(context, config, this, instantTime, partitions).execute(); } @Override - public HoodieWriteMetadata> upsertPrepped(HoodieEngineContext context, String instantTime, - JavaRDD> preppedRecords) { + public HoodieWriteMetadata> upsertPrepped(HoodieEngineContext context, String instantTime, + HoodieData> preppedRecords) { return new SparkUpsertPreppedCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, preppedRecords).execute(); } @Override - public HoodieWriteMetadata> insertPrepped(HoodieEngineContext context, String instantTime, - JavaRDD> preppedRecords) { + public HoodieWriteMetadata> insertPrepped(HoodieEngineContext context, String instantTime, + HoodieData> preppedRecords) { return new SparkInsertPreppedCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, preppedRecords).execute(); } @Override - public HoodieWriteMetadata> bulkInsertPrepped(HoodieEngineContext context, String instantTime, - JavaRDD> preppedRecords, Option>>> userDefinedBulkInsertPartitioner) { + public HoodieWriteMetadata> bulkInsertPrepped(HoodieEngineContext context, String instantTime, + HoodieData> preppedRecords, Option userDefinedBulkInsertPartitioner) { return new SparkBulkInsertPreppedCommitActionExecutor((HoodieSparkEngineContext) context, config, this, instantTime, preppedRecords, userDefinedBulkInsertPartitioner).execute(); } @Override - public HoodieWriteMetadata insertOverwrite(HoodieEngineContext context, String instantTime, JavaRDD> records) { + public HoodieWriteMetadata insertOverwrite(HoodieEngineContext context, String instantTime, HoodieData> records) { return new SparkInsertOverwriteCommitActionExecutor(context, config, this, instantTime, records).execute(); } @Override - public HoodieWriteMetadata> insertOverwriteTable(HoodieEngineContext context, String instantTime, JavaRDD> records) { + public HoodieWriteMetadata> insertOverwriteTable(HoodieEngineContext context, String instantTime, HoodieData> records) { return new SparkInsertOverwriteTableCommitActionExecutor(context, config, this, instantTime, records).execute(); } @@ -235,7 +235,7 @@ public Option scheduleCompaction(HoodieEngineContext conte } @Override - public HoodieWriteMetadata> compact( + public HoodieWriteMetadata> compact( HoodieEngineContext context, String compactionInstantTime) { throw new HoodieNotSupportedException("Compaction is not supported on a CopyOnWrite table"); } @@ -248,20 +248,20 @@ public Option scheduleClustering(HoodieEngineContext conte } @Override - public HoodieWriteMetadata> cluster(HoodieEngineContext context, + public HoodieWriteMetadata> cluster(HoodieEngineContext context, String clusteringInstantTime) { return new SparkExecuteClusteringCommitActionExecutor<>(context, config, this, clusteringInstantTime).execute(); } @Override - public HoodieBootstrapWriteMetadata> bootstrap(HoodieEngineContext context, Option> extraMetadata) { + 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 RestorePlanActionExecutor<>(context, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute(); - new CopyOnWriteRestoreActionExecutor(context, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute(); + new CopyOnWriteRestoreActionExecutor<>(context, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute(); } @Override @@ -292,7 +292,7 @@ protected Iterator> handleUpdateInternal(HoodieMergeHandle> handleInsert( @Override public HoodieCleanMetadata clean(HoodieEngineContext context, String cleanInstantTime, boolean skipLocking) { - return new CleanActionExecutor(context, config, this, cleanInstantTime, skipLocking).execute(); + return new CleanActionExecutor<>(context, config, this, cleanInstantTime, skipLocking).execute(); } @Override public HoodieRollbackMetadata rollback(HoodieEngineContext context, String rollbackInstantTime, HoodieInstant commitInstant, boolean deleteInstants, boolean skipLocking) { - return new CopyOnWriteRollbackActionExecutor((HoodieSparkEngineContext) context, config, this, rollbackInstantTime, commitInstant, + return new CopyOnWriteRollbackActionExecutor<>(context, config, this, rollbackInstantTime, commitInstant, deleteInstants, skipLocking).execute(); } @Override public HoodieSavepointMetadata savepoint(HoodieEngineContext context, String instantToSavepoint, String user, String comment) { - return new SavepointActionExecutor(context, config, this, instantToSavepoint, user, comment).execute(); + return new SavepointActionExecutor<>(context, config, this, instantToSavepoint, user, comment).execute(); } @Override public HoodieRestoreMetadata restore(HoodieEngineContext context, String restoreInstantTime, String instantToRestore) { - return new CopyOnWriteRestoreActionExecutor(context, config, this, restoreInstantTime, instantToRestore).execute(); + return new CopyOnWriteRestoreActionExecutor<>(context, config, this, restoreInstantTime, instantToRestore).execute(); } @Override public Option scheduleRestore(HoodieEngineContext context, String restoreInstantTime, String instantToRestore) { - return new RestorePlanActionExecutor(context, config, this, restoreInstantTime, instantToRestore).execute(); + return new RestorePlanActionExecutor<>(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 index 334efa7fc91f4..efc667af297be 100644 --- 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 @@ -24,6 +24,7 @@ import org.apache.hudi.avro.model.HoodieRollbackPlan; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -31,14 +32,13 @@ 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.bootstrap.SparkBootstrapDeltaCommitActionExecutor; import org.apache.hudi.table.action.compact.HoodieSparkMergeOnReadTableCompactor; import org.apache.hudi.table.action.compact.RunCompactionActionExecutor; import org.apache.hudi.table.action.compact.ScheduleCompactionActionExecutor; @@ -54,8 +54,6 @@ import org.apache.hudi.table.action.rollback.MergeOnReadRollbackActionExecutor; import org.apache.hudi.table.action.rollback.RestorePlanActionExecutor; -import org.apache.spark.api.java.JavaRDD; - import java.util.List; import java.util.Map; @@ -87,72 +85,72 @@ public boolean isTableServiceAction(String actionType) { } @Override - public HoodieWriteMetadata> upsert(HoodieEngineContext context, String instantTime, JavaRDD> records) { + public HoodieWriteMetadata> upsert(HoodieEngineContext context, String instantTime, HoodieData> records) { return new SparkUpsertDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, records).execute(); } @Override - public HoodieWriteMetadata> insert(HoodieEngineContext context, String instantTime, JavaRDD> records) { + public HoodieWriteMetadata> insert(HoodieEngineContext context, String instantTime, HoodieData> 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, + public HoodieWriteMetadata> bulkInsert(HoodieEngineContext context, String instantTime, HoodieData> records, + Option userDefinedBulkInsertPartitioner) { + return new SparkBulkInsertDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, records, userDefinedBulkInsertPartitioner).execute(); } @Override - public HoodieWriteMetadata> delete(HoodieEngineContext context, String instantTime, JavaRDD keys) { + public HoodieWriteMetadata> delete(HoodieEngineContext context, String instantTime, HoodieData keys) { return new SparkDeleteDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, keys).execute(); } @Override - public HoodieWriteMetadata> upsertPrepped(HoodieEngineContext context, String instantTime, - JavaRDD> preppedRecords) { + public HoodieWriteMetadata> upsertPrepped(HoodieEngineContext context, String instantTime, + HoodieData> preppedRecords) { return new SparkUpsertPreppedDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, preppedRecords).execute(); } @Override - public HoodieWriteMetadata> insertPrepped(HoodieEngineContext context, String instantTime, - JavaRDD> preppedRecords) { + public HoodieWriteMetadata> insertPrepped(HoodieEngineContext context, String instantTime, + HoodieData> preppedRecords) { return new SparkInsertPreppedDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, preppedRecords).execute(); } @Override - public HoodieWriteMetadata> bulkInsertPrepped(HoodieEngineContext context, String instantTime, - JavaRDD> preppedRecords, Option>>> userDefinedBulkInsertPartitioner) { + public HoodieWriteMetadata> bulkInsertPrepped(HoodieEngineContext context, String instantTime, + HoodieData> preppedRecords, Option userDefinedBulkInsertPartitioner) { return new SparkBulkInsertPreppedDeltaCommitActionExecutor((HoodieSparkEngineContext) context, config, this, instantTime, preppedRecords, userDefinedBulkInsertPartitioner).execute(); } @Override public Option scheduleCompaction(HoodieEngineContext context, String instantTime, Option> extraMetadata) { - ScheduleCompactionActionExecutor scheduleCompactionExecutor = new ScheduleCompactionActionExecutor( + ScheduleCompactionActionExecutor scheduleCompactionExecutor = new ScheduleCompactionActionExecutor<>( context, config, this, instantTime, extraMetadata, - new HoodieSparkMergeOnReadTableCompactor()); + new HoodieSparkMergeOnReadTableCompactor<>()); return scheduleCompactionExecutor.execute(); } @Override - public HoodieWriteMetadata> compact( + public HoodieWriteMetadata> compact( HoodieEngineContext context, String compactionInstantTime) { - RunCompactionActionExecutor compactionExecutor = new RunCompactionActionExecutor( - context, config, this, compactionInstantTime, new HoodieSparkMergeOnReadTableCompactor(), - new HoodieSparkCopyOnWriteTable(config, context, getMetaClient())); - return convertMetadata(compactionExecutor.execute()); + RunCompactionActionExecutor compactionExecutor = new RunCompactionActionExecutor<>( + context, config, this, compactionInstantTime, new HoodieSparkMergeOnReadTableCompactor<>(), + new HoodieSparkCopyOnWriteTable<>(config, context, getMetaClient())); + return compactionExecutor.execute(); } @Override - public HoodieBootstrapWriteMetadata> bootstrap(HoodieEngineContext context, Option> extraMetadata) { - return new SparkBootstrapDeltaCommitActionExecutor((HoodieSparkEngineContext) context, config, this, extraMetadata).execute(); + 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 RestorePlanActionExecutor<>(context, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute(); - new MergeOnReadRestoreActionExecutor(context, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute(); + new MergeOnReadRestoreActionExecutor<>(context, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute(); } @Override @@ -169,12 +167,12 @@ public HoodieRollbackMetadata rollback(HoodieEngineContext context, HoodieInstant commitInstant, boolean deleteInstants, boolean skipLocking) { - return new MergeOnReadRollbackActionExecutor(context, config, this, rollbackInstantTime, commitInstant, deleteInstants, skipLocking).execute(); + return new MergeOnReadRollbackActionExecutor<>(context, config, this, rollbackInstantTime, commitInstant, deleteInstants, skipLocking).execute(); } @Override public HoodieRestoreMetadata restore(HoodieEngineContext context, String restoreInstantTime, String instantToRestore) { - return new MergeOnReadRestoreActionExecutor(context, config, this, restoreInstantTime, instantToRestore).execute(); + return new MergeOnReadRestoreActionExecutor<>(context, config, this, restoreInstantTime, instantToRestore).execute(); } @Override 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 index bb8c95d745ab1..ce14d43cfc6e9 100644 --- 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 @@ -36,20 +36,16 @@ import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; -import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.avro.specific.SpecificRecordBase; import org.apache.hadoop.fs.Path; import org.apache.spark.TaskContext; import org.apache.spark.TaskContext$; -import org.apache.spark.api.java.JavaRDD; import java.io.IOException; -import static org.apache.hudi.data.HoodieJavaRDD.getJavaRDD; - public abstract class HoodieSparkTable - extends HoodieTable>, JavaRDD, JavaRDD> { + extends HoodieTable>, HoodieData, HoodieData> { private volatile boolean isMetadataTableExists = false; @@ -81,7 +77,7 @@ public static HoodieSparkTable create(HoodieW HoodieSparkEngineContext context, HoodieTableMetaClient metaClient, boolean refreshTimeline) { - HoodieSparkTable hoodieSparkTable; + HoodieSparkTable hoodieSparkTable; switch (metaClient.getTableType()) { case COPY_ON_WRITE: hoodieSparkTable = new HoodieSparkCopyOnWriteTable<>(config, context, metaClient); @@ -98,11 +94,6 @@ public static HoodieSparkTable create(HoodieW return hoodieSparkTable; } - public static HoodieWriteMetadata> convertMetadata( - HoodieWriteMetadata> metadata) { - return metadata.clone(getJavaRDD(metadata.getWriteStatuses())); - } - @Override protected HoodieIndex getIndex(HoodieWriteConfig config, HoodieEngineContext context) { return SparkHoodieIndexFactory.createIndex(config); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java index a970e8f0f97d3..504da8a722810 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java @@ -28,10 +28,10 @@ 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.client.utils.SparkValidatorUtils; import org.apache.hudi.common.bootstrap.index.BootstrapIndex; import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.BootstrapFileMapping; import org.apache.hudi.common.model.HoodieCommitMetadata; @@ -68,7 +68,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 java.io.IOException; import java.nio.charset.StandardCharsets; @@ -80,10 +79,11 @@ import java.util.Map; import java.util.stream.Collectors; +import static org.apache.hudi.config.HoodieWriteConfig.WRITE_STATUS_STORAGE_LEVEL_VALUE; import static org.apache.hudi.table.action.bootstrap.MetadataBootstrapHandlerFactory.getMetadataHandler; public class SparkBootstrapCommitActionExecutor> - extends BaseCommitActionExecutor>, JavaRDD, JavaRDD, HoodieBootstrapWriteMetadata> { + extends BaseCommitActionExecutor>, HoodieData, HoodieData, HoodieBootstrapWriteMetadata>> { private static final Logger LOG = LogManager.getLogger(SparkBootstrapCommitActionExecutor.class); protected String bootstrapSchema = null; @@ -91,7 +91,7 @@ public class SparkBootstrapCommitActionExecutor public SparkBootstrapCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, - HoodieTable>, JavaRDD, JavaRDD> table, + HoodieTable table, Option> extraMetadata) { super(context, new HoodieWriteConfig.Builder().withProps(config.getProps()) .withAutoCommit(true).withWriteStatusClass(BootstrapWriteStatus.class) @@ -109,7 +109,7 @@ private void validate() { } @Override - public HoodieBootstrapWriteMetadata execute() { + public HoodieBootstrapWriteMetadata> execute() { validate(); try { HoodieTableMetaClient metaClient = table.getMetaClient(); @@ -121,9 +121,9 @@ public HoodieBootstrapWriteMetadata execute() { Map>>> partitionSelections = listAndProcessSourcePartitions(); // First run metadata bootstrap which will auto commit - Option metadataResult = metadataBootstrap(partitionSelections.get(BootstrapMode.METADATA_ONLY)); + Option>> metadataResult = metadataBootstrap(partitionSelections.get(BootstrapMode.METADATA_ONLY)); // if there are full bootstrap to be performed, perform that too - Option fullBootstrapResult = fullBootstrap(partitionSelections.get(BootstrapMode.FULL_RECORD)); + Option>> fullBootstrapResult = fullBootstrap(partitionSelections.get(BootstrapMode.FULL_RECORD)); // Delete the marker directory for the instant WriteMarkersFactory.get(config.getMarkersType(), table, instantTime) .quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism()); @@ -142,7 +142,7 @@ protected String getSchemaToStoreInCommit() { * Perform Metadata Bootstrap. * @param partitionFilesList List of partitions and files within that partitions */ - protected Option metadataBootstrap(List>> partitionFilesList) { + protected Option>> metadataBootstrap(List>> partitionFilesList) { if (null == partitionFilesList || partitionFilesList.isEmpty()) { return Option.empty(); } @@ -155,43 +155,42 @@ protected Option metadataBootstrap(List bootstrapWriteStatuses = runMetadataBootstrap(partitionFilesList); + HoodieData 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) { + private void updateIndexAndCommitIfNeeded(HoodieData writeStatuses, 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())); + writeStatuses.persist(config.getString(WRITE_STATUS_STORAGE_LEVEL_VALUE)); Instant indexStartTime = Instant.now(); // Update the index back - JavaRDD statuses = HoodieJavaRDD.getJavaRDD( - table.getIndex().updateLocation(HoodieJavaRDD.of(writeStatusRDD), context, table)); + HoodieData statuses = table.getIndex().updateLocation(writeStatuses, context, table); result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now())); result.setWriteStatuses(statuses); commitOnAutoCommit(result); } @Override - public HoodieWriteMetadata> execute(JavaRDD> inputRecords) { + public HoodieWriteMetadata> execute(HoodieData> inputRecords) { // NO_OP return null; } @Override - protected void setCommitMetadata(HoodieWriteMetadata> result) { + protected void setCommitMetadata(HoodieWriteMetadata> result) { result.setCommitMetadata(Option.of(new HoodieCommitMetadata())); } @Override - protected void commit(Option> extraMetadata, HoodieWriteMetadata> result) { + 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 = - result.getWriteStatuses().collect().stream() + result.getWriteStatuses().collectAsList().stream() .map(w -> { BootstrapWriteStatus ws = (BootstrapWriteStatus) w; return Pair.of(ws.getBootstrapSourceFileMapping(), ws.getStat()); @@ -214,7 +213,7 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta LOG.info("Committing metadata bootstrap !!"); } - protected void commit(Option> extraMetadata, HoodieWriteMetadata> result, List stats) { + 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 @@ -253,7 +252,7 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta * Perform Full Bootstrap. * @param partitionFilesList List of partitions and files within that partitions */ - protected Option fullBootstrap(List>> partitionFilesList) { + protected Option>> fullBootstrap(List>> partitionFilesList) { if (null == partitionFilesList || partitionFilesList.isEmpty()) { return Option.empty(); } @@ -271,10 +270,10 @@ protected Option fullBootstrap(List getBulkInsertActionExecutor(JavaRDD inputRecordsRDD) { + protected BaseSparkCommitActionExecutor getBulkInsertActionExecutor(HoodieData inputRecordsRDD) { return new SparkBulkInsertCommitActionExecutor((HoodieSparkEngineContext) context, new HoodieWriteConfig.Builder().withProps(config.getProps()) .withSchema(bootstrapSchema).build(), table, HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS, inputRecordsRDD, Option.empty(), extraMetadata); @@ -310,10 +309,9 @@ private Map>>> listAndPr .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); } - private JavaRDD runMetadataBootstrap(List>> partitions) { - JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); + private HoodieData runMetadataBootstrap(List>> partitions) { if (null == partitions || partitions.isEmpty()) { - return jsc.emptyRDD(); + return context.emptyHoodieData(); } TypedProperties properties = new TypedProperties(); @@ -336,7 +334,8 @@ private JavaRDD runMetadataBootstrap(List getMetadataHandler(config, table, partitionFsPair.getRight().getRight()).runMetadataBootstrap(partitionFsPair.getLeft(), partitionFsPair.getRight().getLeft(), keyGenerator)); } @@ -352,7 +351,7 @@ protected Iterator> handleUpdate(String partitionPath, String } @Override - protected void runPrecommitValidators(HoodieWriteMetadata> writeMetadata) { + protected void runPrecommitValidators(HoodieWriteMetadata> writeMetadata) { SparkValidatorUtils.runValidators(config, writeMetadata, context, table, instantTime); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapDeltaCommitActionExecutor.java index 59f86662b7c0c..d712ca430b6f7 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapDeltaCommitActionExecutor.java @@ -18,9 +18,8 @@ package org.apache.hudi.table.action.bootstrap; -import java.util.Map; - import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -29,7 +28,8 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor; import org.apache.hudi.table.action.deltacommit.SparkBulkInsertDeltaCommitActionExecutor; -import org.apache.spark.api.java.JavaRDD; + +import java.util.Map; public class SparkBootstrapDeltaCommitActionExecutor> extends SparkBootstrapCommitActionExecutor { @@ -41,7 +41,7 @@ public SparkBootstrapDeltaCommitActionExecutor(HoodieSparkEngineContext context, } @Override - protected BaseSparkCommitActionExecutor getBulkInsertActionExecutor(JavaRDD inputRecordsRDD) { + protected BaseSparkCommitActionExecutor getBulkInsertActionExecutor(HoodieData 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/cluster/SparkExecuteClusteringCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java index 594a910428aad..7d2a4c0baabe3 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java @@ -28,14 +28,11 @@ import org.apache.hudi.common.util.ClusteringUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.exception.HoodieClusteringException; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor; -import org.apache.spark.api.java.JavaRDD; - public class SparkExecuteClusteringCommitActionExecutor> extends BaseSparkCommitActionExecutor { @@ -52,10 +49,8 @@ public SparkExecuteClusteringCommitActionExecutor(HoodieEngineContext context, } @Override - public HoodieWriteMetadata> execute() { - HoodieWriteMetadata> writeMetadata = executeClustering(clusteringPlan); - JavaRDD transformedWriteStatuses = HoodieJavaRDD.getJavaRDD(writeMetadata.getWriteStatuses()); - return writeMetadata.clone(transformedWriteStatuses); + public HoodieWriteMetadata> execute() { + return executeClustering(clusteringPlan); } @Override diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java index ba3b0be1641ee..ade550897765a 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/BaseSparkCommitActionExecutor.java @@ -19,8 +19,8 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.utils.SparkMemoryUtils; import org.apache.hudi.client.utils.SparkValidatorUtils; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFileGroupId; @@ -37,6 +37,7 @@ import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieJavaPairRDD; import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.exception.HoodieIOException; @@ -54,13 +55,13 @@ import org.apache.hudi.table.WorkloadStat; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.cluster.strategy.UpdateStrategy; + 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.storage.StorageLevel; -import scala.Tuple2; import java.io.IOException; import java.io.Serializable; @@ -76,10 +77,13 @@ import java.util.Set; import java.util.stream.Collectors; +import scala.Tuple2; + import static org.apache.hudi.common.util.ClusteringUtils.getAllFileGroupsInPendingClusteringPlans; +import static org.apache.hudi.config.HoodieWriteConfig.WRITE_STATUS_STORAGE_LEVEL_VALUE; public abstract class BaseSparkCommitActionExecutor extends - BaseCommitActionExecutor>, JavaRDD, JavaRDD, HoodieWriteMetadata> { + BaseCommitActionExecutor>, HoodieData, HoodieData, HoodieWriteMetadata>> { private static final Logger LOG = LogManager.getLogger(BaseSparkCommitActionExecutor.class); protected final Option keyGeneratorOpt; @@ -97,7 +101,7 @@ public BaseSparkCommitActionExecutor(HoodieEngineContext context, HoodieTable table, String instantTime, WriteOperationType operationType, - Option extraMetadata) { + Option> extraMetadata) { super(context, config, table, instantTime, operationType, extraMetadata); try { keyGeneratorOpt = config.populateMetaFields() @@ -108,14 +112,14 @@ public BaseSparkCommitActionExecutor(HoodieEngineContext context, } } - private JavaRDD> clusteringHandleUpdate(JavaRDD> inputRecordsRDD) { + private HoodieData> clusteringHandleUpdate(HoodieData> inputRecords) { context.setJobStatus(this.getClass().getSimpleName(), "Handling updates which are under clustering"); Set fileGroupsInPendingClustering = - table.getFileSystemView().getFileGroupsInPendingClustering().map(entry -> entry.getKey()).collect(Collectors.toSet()); - UpdateStrategy updateStrategy = (UpdateStrategy) ReflectionUtils + table.getFileSystemView().getFileGroupsInPendingClustering().map(Pair::getKey).collect(Collectors.toSet()); + UpdateStrategy>> updateStrategy = (UpdateStrategy>>) ReflectionUtils .loadClass(config.getClusteringUpdatesStrategyClass(), this.context, fileGroupsInPendingClustering); - Pair>, Set> recordsAndPendingClusteringFileGroups = - (Pair>, Set>) updateStrategy.handleUpdate(inputRecordsRDD); + Pair>, Set> recordsAndPendingClusteringFileGroups = + updateStrategy.handleUpdate(inputRecords); Set fileGroupsWithUpdatesAndPendingClustering = recordsAndPendingClusteringFileGroups.getRight(); if (fileGroupsWithUpdatesAndPendingClustering.isEmpty()) { return recordsAndPendingClusteringFileGroups.getLeft(); @@ -138,20 +142,20 @@ private JavaRDD> clusteringHandleUpdate(JavaRDD> } @Override - public HoodieWriteMetadata> execute(JavaRDD> inputRecordsRDD) { - HoodieWriteMetadata> result = new HoodieWriteMetadata<>(); + public HoodieWriteMetadata> execute(HoodieData> inputRecords) { // 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()) { - inputRecordsRDD.persist(StorageLevel.MEMORY_AND_DISK_SER()); + JavaRDD> inputRDD = HoodieJavaRDD.getJavaRDD(inputRecords); + if (inputRDD.getStorageLevel() == StorageLevel.NONE()) { + inputRDD.persist(StorageLevel.MEMORY_AND_DISK_SER()); } else { - LOG.info("RDD PreppedRecords was persisted at: " + inputRecordsRDD.getStorageLevel()); + LOG.info("RDD PreppedRecords was persisted at: " + inputRDD.getStorageLevel()); } WorkloadProfile workloadProfile = null; if (isWorkloadProfileNeeded()) { context.setJobStatus(this.getClass().getSimpleName(), "Building workload profile"); - workloadProfile = new WorkloadProfile(buildProfile(inputRecordsRDD), operationType, table.getIndex().canIndexLogFiles()); + workloadProfile = new WorkloadProfile(buildProfile(inputRecords), operationType, table.getIndex().canIndexLogFiles()); LOG.info("Input workload profile :" + workloadProfile); } @@ -162,30 +166,23 @@ public HoodieWriteMetadata> execute(JavaRDD } // handle records update with clustering - JavaRDD> inputRecordsRDDWithClusteringUpdate = clusteringHandleUpdate(inputRecordsRDD); + HoodieData> inputRecordsWithClusteringUpdate = clusteringHandleUpdate(inputRecords); context.setJobStatus(this.getClass().getSimpleName(), "Doing partition and writing data"); - JavaRDD> partitionedRecords = partition(inputRecordsRDDWithClusteringUpdate, partitioner); - JavaRDD writeStatusRDD = partitionedRecords.mapPartitionsWithIndex((partition, recordItr) -> { - if (WriteOperationType.isChangingRecords(operationType)) { - return handleUpsertPartition(instantTime, partition, recordItr, partitioner); - } else { - return handleInsertPartition(instantTime, partition, recordItr, partitioner); - } - }, true).flatMap(List::iterator); - - updateIndexAndCommitIfNeeded(writeStatusRDD, result); + HoodieData writeStatuses = mapPartitionsAsRDD(inputRecordsWithClusteringUpdate, partitioner); + HoodieWriteMetadata> result = new HoodieWriteMetadata<>(); + updateIndexAndCommitIfNeeded(writeStatuses, result); return result; } - private Pair, WorkloadStat> buildProfile(JavaRDD> inputRecordsRDD) { + private Pair, WorkloadStat> buildProfile(HoodieData> inputRecords) { 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<>( + Map>, Long> partitionLocationCounts = inputRecords + .mapToPair(record -> Pair.of( new Tuple2<>(record.getPartitionPath(), Option.ofNullable(record.getCurrentLocation())), record)) .countByKey(); @@ -223,9 +220,9 @@ protected Partitioner getPartitioner(WorkloadProfile profile) { } } - private JavaRDD> partition(JavaRDD> dedupedRecords, Partitioner partitioner) { - JavaPairRDD>, HoodieRecord> mappedRDD = dedupedRecords.mapToPair( - record -> new Tuple2<>(new Tuple2<>(record.getKey(), Option.ofNullable(record.getCurrentLocation())), record)); + private HoodieData mapPartitionsAsRDD(HoodieData> dedupedRecords, Partitioner partitioner) { + JavaPairRDD>, HoodieRecord> mappedRDD = HoodieJavaPairRDD.getJavaPairRDD( + dedupedRecords.mapToPair(record -> Pair.of(new Tuple2<>(record.getKey(), Option.ofNullable(record.getCurrentLocation())), record))); JavaPairRDD>, HoodieRecord> partitionedRDD; if (table.requireSortedRecords()) { @@ -242,24 +239,28 @@ record -> new Tuple2<>(new Tuple2<>(record.getKey(), Option.ofNullable(record.ge // Partition only partitionedRDD = mappedRDD.partitionBy(partitioner); } - - return partitionedRDD.map(Tuple2::_2); + return HoodieJavaRDD.of(partitionedRDD.map(Tuple2::_2).mapPartitionsWithIndex((partition, recordItr) -> { + if (WriteOperationType.isChangingRecords(operationType)) { + return handleUpsertPartition(instantTime, partition, recordItr, partitioner); + } else { + return handleInsertPartition(instantTime, partition, recordItr, partitioner); + } + }, true).flatMap(List::iterator)); } - protected JavaRDD updateIndex(JavaRDD writeStatusRDD, HoodieWriteMetadata result) { + protected HoodieData updateIndex(HoodieData writeStatuses, 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())); + writeStatuses.persist(config.getString(WRITE_STATUS_STORAGE_LEVEL_VALUE)); Instant indexStartTime = Instant.now(); // Update the index back - JavaRDD statuses = HoodieJavaRDD.getJavaRDD( - table.getIndex().updateLocation(HoodieJavaRDD.of(writeStatusRDD), context, table)); + HoodieData statuses = table.getIndex().updateLocation(writeStatuses, context, table); result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now())); result.setWriteStatuses(statuses); return statuses; } - protected void updateIndexAndCommitIfNeeded(JavaRDD writeStatusRDD, HoodieWriteMetadata result) { + protected void updateIndexAndCommitIfNeeded(HoodieData writeStatusRDD, HoodieWriteMetadata> result) { updateIndex(writeStatusRDD, result); result.setPartitionToReplaceFileIds(getPartitionToReplacedFileIds(result)); commitOnAutoCommit(result); @@ -271,19 +272,19 @@ protected String getCommitActionType() { } @Override - protected void setCommitMetadata(HoodieWriteMetadata> result) { - result.setCommitMetadata(Option.of(CommitUtils.buildMetadata(result.getWriteStatuses().map(WriteStatus::getStat).collect(), + protected void setCommitMetadata(HoodieWriteMetadata> result) { + result.setCommitMetadata(Option.of(CommitUtils.buildMetadata(result.getWriteStatuses().map(WriteStatus::getStat).collectAsList(), result.getPartitionToReplaceFileIds(), extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType()))); } @Override - protected void commit(Option> extraMetadata, HoodieWriteMetadata> result) { + protected void commit(Option> extraMetadata, HoodieWriteMetadata> result) { context.setJobStatus(this.getClass().getSimpleName(), "Commit write status collect"); - commit(extraMetadata, result, result.getWriteStatuses().map(WriteStatus::getStat).collect()); + commit(extraMetadata, result, result.getWriteStatuses().map(WriteStatus::getStat).collectAsList()); } - 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 + ", operation Type " + operationType); result.setCommitted(true); @@ -304,7 +305,7 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta } } - protected Map> getPartitionToReplacedFileIds(HoodieWriteMetadata> writeStatuses) { + protected Map> getPartitionToReplacedFileIds(HoodieWriteMetadata> writeStatuses) { return Collections.emptyMap(); } @@ -341,20 +342,20 @@ public Iterator> handleUpdate(String partitionPath, String fil // 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(); + return Collections.emptyIterator(); } // these are updates HoodieMergeHandle upsertHandle = getUpdateHandle(partitionPath, fileId, recordItr); return handleUpdateInternal(upsertHandle, fileId); } - protected Iterator> handleUpdateInternal(HoodieMergeHandle upsertHandle, String 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 { - SparkMergeHelper.newInstance().runMerge(table, upsertHandle); + HoodieMergeHelper.newInstance().runMerge(table, upsertHandle); } // TODO(vc): This needs to be revisited @@ -383,9 +384,9 @@ public Iterator> handleInsert(String idPfx, Iterator) Collections.EMPTY_LIST).iterator(); + return Collections.emptyIterator(); } - return new SparkLazyInsertIterable(recordItr, true, config, instantTime, table, idPfx, + return new SparkLazyInsertIterable<>(recordItr, true, config, instantTime, table, idPfx, taskContextSupplier, new CreateHandleFactory<>()); } @@ -393,7 +394,7 @@ 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); + return new UpsertPartitioner<>(profile, context, table, config); } public Partitioner getInsertPartitioner(WorkloadProfile profile) { @@ -407,7 +408,7 @@ public Partitioner getLayoutPartitioner(WorkloadProfile profile, String layoutPa } @Override - protected void runPrecommitValidators(HoodieWriteMetadata> writeMetadata) { + protected void runPrecommitValidators(HoodieWriteMetadata> writeMetadata) { SparkValidatorUtils.runValidators(config, writeMetadata, context, table, instantTime); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertCommitActionExecutor.java index f4f1d3ad06ccf..f4b01c887b068 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertCommitActionExecutor.java @@ -20,6 +20,7 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -30,24 +31,22 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.spark.api.java.JavaRDD; - import java.util.Map; public class SparkBulkInsertCommitActionExecutor> extends BaseSparkCommitActionExecutor { - private final JavaRDD> inputRecordsRDD; - private final Option>>> bulkInsertPartitioner; + private final HoodieData> inputRecordsRDD; + private final Option bulkInsertPartitioner; public SparkBulkInsertCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD, - Option>>> bulkInsertPartitioner) { + String instantTime, HoodieData> inputRecordsRDD, + Option bulkInsertPartitioner) { this(context, config, table, instantTime, inputRecordsRDD, bulkInsertPartitioner, Option.empty()); } public SparkBulkInsertCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD, - Option>>> bulkInsertPartitioner, + String instantTime, HoodieData> inputRecordsRDD, + Option bulkInsertPartitioner, Option> extraMetadata) { super(context, config, table, instantTime, WriteOperationType.BULK_INSERT, extraMetadata); this.inputRecordsRDD = inputRecordsRDD; @@ -55,7 +54,7 @@ public SparkBulkInsertCommitActionExecutor(HoodieSparkEngineContext context, Hoo } @Override - public HoodieWriteMetadata> execute() { + public HoodieWriteMetadata> execute() { try { return SparkBulkInsertHelper.newInstance().bulkInsert(inputRecordsRDD, instantTime, table, config, this, true, bulkInsertPartitioner); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java index d0c5ddef5e71d..38e38101b0d02 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java @@ -19,6 +19,7 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -26,6 +27,7 @@ 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.data.HoodieJavaRDD; import org.apache.hudi.execution.bulkinsert.BulkInsertInternalPartitionerFactory; import org.apache.hudi.execution.bulkinsert.BulkInsertMapFunction; import org.apache.hudi.io.CreateHandleFactory; @@ -46,28 +48,28 @@ * @param */ @SuppressWarnings("checkstyle:LineLength") -public class SparkBulkInsertHelper extends BaseBulkInsertHelper>, - JavaRDD, JavaRDD, R> { +public class SparkBulkInsertHelper extends BaseBulkInsertHelper>, + HoodieData, HoodieData, R> { private SparkBulkInsertHelper() { } private static class BulkInsertHelperHolder { - private static final SparkBulkInsertHelper SPARK_BULK_INSERT_HELPER = new SparkBulkInsertHelper(); + private static final SparkBulkInsertHelper HOODIE_BULK_INSERT_HELPER = new SparkBulkInsertHelper<>(); } public static SparkBulkInsertHelper newInstance() { - return BulkInsertHelperHolder.SPARK_BULK_INSERT_HELPER; + return BulkInsertHelperHolder.HOODIE_BULK_INSERT_HELPER; } @Override - public HoodieWriteMetadata> bulkInsert(final JavaRDD> inputRecords, - final String instantTime, - final HoodieTable>, JavaRDD, JavaRDD> table, - final HoodieWriteConfig config, - final BaseCommitActionExecutor>, JavaRDD, JavaRDD, R> executor, - final boolean performDedupe, - final Option>>> userDefinedBulkInsertPartitioner) { + public HoodieWriteMetadata> bulkInsert(final HoodieData> inputRecords, + final String instantTime, + final HoodieTable>, HoodieData, HoodieData> table, + final HoodieWriteConfig config, + final BaseCommitActionExecutor>, HoodieData, HoodieData, R> executor, + final boolean performDedupe, + final Option userDefinedBulkInsertPartitioner) { HoodieWriteMetadata result = new HoodieWriteMetadata(); //transition bulk_insert state to inflight @@ -75,7 +77,7 @@ public HoodieWriteMetadata> bulkInsert(final JavaRDD writeStatuses = + HoodieData writeStatuses = bulkInsert(inputRecords, instantTime, table, config, performDedupe, userDefinedBulkInsertPartitioner, false, config.getBulkInsertShuffleParallelism(), new CreateHandleFactory(false)); //update index ((BaseSparkCommitActionExecutor) executor).updateIndexAndCommitIfNeeded(writeStatuses, result); @@ -83,39 +85,40 @@ public HoodieWriteMetadata> bulkInsert(final JavaRDD bulkInsert(JavaRDD> inputRecords, - String instantTime, - HoodieTable>, JavaRDD, JavaRDD> table, - HoodieWriteConfig config, - boolean performDedupe, - Option>>> userDefinedBulkInsertPartitioner, - boolean useWriterSchema, - int parallelism, - WriteHandleFactory writeHandleFactory) { + public HoodieData bulkInsert(HoodieData> inputRecords, + String instantTime, + HoodieTable>, HoodieData, HoodieData> table, + HoodieWriteConfig config, + boolean performDedupe, + Option userDefinedBulkInsertPartitioner, + boolean useWriterSchema, + int parallelism, + WriteHandleFactory writeHandleFactory) { // De-dupe/merge if needed - JavaRDD> dedupedRecords = inputRecords; + HoodieData> dedupedRecords = inputRecords; if (performDedupe) { - dedupedRecords = (JavaRDD>) SparkWriteHelper.newInstance().combineOnCondition(config.shouldCombineBeforeInsert(), inputRecords, + dedupedRecords = (HoodieData>) HoodieWriteHelper.newInstance().combineOnCondition(config.shouldCombineBeforeInsert(), inputRecords, parallelism, table); } - final JavaRDD> repartitionedRecords; + final HoodieData> repartitionedRecords; BulkInsertPartitioner partitioner = userDefinedBulkInsertPartitioner.isPresent() ? userDefinedBulkInsertPartitioner.get() : BulkInsertInternalPartitionerFactory.get(config.getBulkInsertSortMode()); - repartitionedRecords = (JavaRDD>) partitioner.repartitionRecords(dedupedRecords, parallelism); + // only JavaRDD is supported for Spark partitioner, but it is not enforced by BulkInsertPartitioner API. To improve this, TODO HUDI-3463 + repartitionedRecords = HoodieJavaRDD.of((JavaRDD>) partitioner.repartitionRecords(HoodieJavaRDD.getJavaRDD(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()); - JavaRDD writeStatusRDD = repartitionedRecords - .mapPartitionsWithIndex(new BulkInsertMapFunction(instantTime, + JavaRDD writeStatusRDD = HoodieJavaRDD.getJavaRDD(repartitionedRecords) + .mapPartitionsWithIndex(new BulkInsertMapFunction<>(instantTime, partitioner.arePartitionRecordsSorted(), config, table, fileIDPrefixes, useWriterSchema, writeHandleFactory), true) .flatMap(List::iterator); - return writeStatusRDD; + return HoodieJavaRDD.of(writeStatusRDD); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertPreppedCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertPreppedCommitActionExecutor.java index 28d8cb0b26422..8862981c2a2b7 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertPreppedCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertPreppedCommitActionExecutor.java @@ -20,6 +20,7 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -30,25 +31,23 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.spark.api.java.JavaRDD; - public class SparkBulkInsertPreppedCommitActionExecutor> extends BaseSparkCommitActionExecutor { - private final JavaRDD> preppedInputRecordRdd; - private final Option>>> userDefinedBulkInsertPartitioner; + private final HoodieData> preppedInputRecordRdd; + private final Option userDefinedBulkInsertPartitioner; public SparkBulkInsertPreppedCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> preppedInputRecordRdd, - Option>>> userDefinedBulkInsertPartitioner) { + String instantTime, HoodieData> 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 SparkBulkInsertHelper.newInstance().bulkInsert(preppedInputRecordRdd, instantTime, table, config, this, false, userDefinedBulkInsertPartitioner); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteCommitActionExecutor.java index 997c7bf2376e3..a6fc996b71c31 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeleteCommitActionExecutor.java @@ -20,29 +20,28 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; 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.HoodieWriteMetadata; -import org.apache.spark.api.java.JavaRDD; public class SparkDeleteCommitActionExecutor> extends BaseSparkCommitActionExecutor { - private final JavaRDD keys; + private final HoodieData keys; public SparkDeleteCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD keys) { + String instantTime, HoodieData keys) { super(context, config, table, instantTime, WriteOperationType.DELETE); this.keys = keys; } @Override - public HoodieWriteMetadata> execute() { - return SparkDeleteHelper.newInstance().execute(instantTime, keys, context, config, table, this); + public HoodieWriteMetadata> execute() { + return HoodieDeleteHelper.newInstance().execute(instantTime, keys, context, config, table, this); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeletePartitionCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeletePartitionCommitActionExecutor.java index 90bcdc9b9c141..b31eb7b96d948 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeletePartitionCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeletePartitionCommitActionExecutor.java @@ -19,20 +19,18 @@ 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.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieJavaPairRDD; 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 scala.Tuple2; import java.time.Duration; import java.util.HashMap; @@ -51,16 +49,15 @@ public SparkDeletePartitionCommitActionExecutor(HoodieEngineContext context, } @Override - public HoodieWriteMetadata> execute() { - JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); + public HoodieWriteMetadata> execute() { HoodieTimer timer = new HoodieTimer().startTimer(); - Map> partitionToReplaceFileIds = jsc.parallelize(partitions, partitions.size()).distinct() - .mapToPair(partitionPath -> new Tuple2<>(partitionPath, getAllExistingFileIds(partitionPath))).collectAsMap(); - HoodieWriteMetadata result = new HoodieWriteMetadata(); + context.setJobStatus(this.getClass().getSimpleName(), "Gather all file ids from all deleting partitions."); + Map> partitionToReplaceFileIds = HoodieJavaPairRDD.getJavaPairRDD(context.parallelize(partitions).distinct() + .mapToPair(partitionPath -> Pair.of(partitionPath, getAllExistingFileIds(partitionPath)))).collectAsMap(); + HoodieWriteMetadata> result = new HoodieWriteMetadata<>(); result.setPartitionToReplaceFileIds(partitionToReplaceFileIds); result.setIndexUpdateDuration(Duration.ofMillis(timer.endTimer())); - - result.setWriteStatuses(jsc.emptyRDD()); + result.setWriteStatuses(context.emptyHoodieData()); this.saveWorkloadProfileMetadataToInflight(new WorkloadProfile(Pair.of(new HashMap<>(), new WorkloadStat())), instantTime); this.commitOnAutoCommit(result); return result; diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertCommitActionExecutor.java index ba91fe1607916..479b51322ff32 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertCommitActionExecutor.java @@ -20,30 +20,29 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; 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.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; - import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.spark.api.java.JavaRDD; public class SparkInsertCommitActionExecutor> extends BaseSparkCommitActionExecutor { - private final JavaRDD> inputRecordsRDD; + private final HoodieData> inputRecordsRDD; public SparkInsertCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD) { + String instantTime, HoodieData> inputRecordsRDD) { super(context, config, table, instantTime, WriteOperationType.INSERT); this.inputRecordsRDD = inputRecordsRDD; } @Override - public HoodieWriteMetadata> execute() { - return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, + public HoodieWriteMetadata> execute() { + return HoodieWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, operationType); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java index 7a3549c9e9c41..518063ed34186 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteCommitActionExecutor.java @@ -19,18 +19,21 @@ package org.apache.hudi.table.action.commit; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.FileSlice; 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.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieJavaPairRDD; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.WorkloadProfile; import org.apache.hudi.table.action.HoodieWriteMetadata; + import org.apache.spark.Partitioner; -import org.apache.spark.api.java.JavaRDD; -import scala.Tuple2; import java.util.List; import java.util.Map; @@ -39,25 +42,25 @@ public class SparkInsertOverwriteCommitActionExecutor> extends BaseSparkCommitActionExecutor { - private final JavaRDD> inputRecordsRDD; + private final HoodieData> inputRecordsRDD; public SparkInsertOverwriteCommitActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD) { + String instantTime, HoodieData> inputRecordsRDD) { this(context, config, table, instantTime, inputRecordsRDD, WriteOperationType.INSERT_OVERWRITE); } public SparkInsertOverwriteCommitActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD, + String instantTime, HoodieData> inputRecordsRDD, WriteOperationType writeOperationType) { super(context, config, table, instantTime, writeOperationType); this.inputRecordsRDD = inputRecordsRDD; } @Override - public HoodieWriteMetadata> execute() { - return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, + public HoodieWriteMetadata> execute() { + return HoodieWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, operationType); } @@ -74,13 +77,13 @@ protected String getCommitActionType() { } @Override - protected Map> getPartitionToReplacedFileIds(HoodieWriteMetadata> writeMetadata) { - return writeMetadata.getWriteStatuses().map(status -> status.getStat().getPartitionPath()).distinct().mapToPair(partitionPath -> - new Tuple2<>(partitionPath, getAllExistingFileIds(partitionPath))).collectAsMap(); + protected Map> getPartitionToReplacedFileIds(HoodieWriteMetadata> writeMetadata) { + return HoodieJavaPairRDD.getJavaPairRDD(writeMetadata.getWriteStatuses().map(status -> status.getStat().getPartitionPath()).distinct().mapToPair(partitionPath -> + Pair.of(partitionPath, getAllExistingFileIds(partitionPath)))).collectAsMap(); } protected List getAllExistingFileIds(String partitionPath) { // because new commit is not complete. it is safe to mark all existing file Ids as old files - return table.getSliceView().getLatestFileSlices(partitionPath).map(fg -> fg.getFileId()).distinct().collect(Collectors.toList()); + return table.getSliceView().getLatestFileSlices(partitionPath).map(FileSlice::getFileId).distinct().collect(Collectors.toList()); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitActionExecutor.java index f7c98d5373360..93d0a8124c4ee 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertOverwriteTableCommitActionExecutor.java @@ -19,20 +19,19 @@ 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.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; 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.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieJavaPairRDD; 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 scala.Tuple2; -import java.util.HashMap; +import java.util.Collections; import java.util.List; import java.util.Map; @@ -41,21 +40,18 @@ public class SparkInsertOverwriteTableCommitActionExecutor> inputRecordsRDD) { + String instantTime, HoodieData> inputRecordsRDD) { super(context, config, table, instantTime, inputRecordsRDD, WriteOperationType.INSERT_OVERWRITE_TABLE); } @Override - protected Map> getPartitionToReplacedFileIds(HoodieWriteMetadata> writeMetadata) { - Map> partitionToExistingFileIds = new HashMap<>(); + protected Map> getPartitionToReplacedFileIds(HoodieWriteMetadata> writeMetadata) { List partitionPaths = FSUtils.getAllPartitionPaths(context, config.getMetadataConfig(), table.getMetaClient().getBasePath()); - JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); - if (partitionPaths != null && partitionPaths.size() > 0) { - context.setJobStatus(this.getClass().getSimpleName(), "Getting ExistingFileIds of all partitions"); - JavaRDD partitionPathRdd = jsc.parallelize(partitionPaths, partitionPaths.size()); - partitionToExistingFileIds = partitionPathRdd.mapToPair( - partitionPath -> new Tuple2<>(partitionPath, getAllExistingFileIds(partitionPath))).collectAsMap(); + if (partitionPaths == null || partitionPaths.isEmpty()) { + return Collections.emptyMap(); } - return partitionToExistingFileIds; + context.setJobStatus(this.getClass().getSimpleName(), "Getting ExistingFileIds of all partitions"); + return HoodieJavaPairRDD.getJavaPairRDD(context.parallelize(partitionPaths, partitionPaths.size()).mapToPair( + partitionPath -> Pair.of(partitionPath, getAllExistingFileIds(partitionPath)))).collectAsMap(); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertPreppedCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertPreppedCommitActionExecutor.java index 400147bb8fe70..ff1a7e2b9beeb 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertPreppedCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkInsertPreppedCommitActionExecutor.java @@ -20,29 +20,28 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; 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.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; - import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.spark.api.java.JavaRDD; public class SparkInsertPreppedCommitActionExecutor> extends BaseSparkCommitActionExecutor { - private final JavaRDD> preppedRecords; + private final HoodieData> preppedRecords; public SparkInsertPreppedCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> preppedRecords) { + String instantTime, HoodieData> preppedRecords) { super(context, config, table, instantTime, WriteOperationType.INSERT_PREPPED); this.preppedRecords = preppedRecords; } @Override - public HoodieWriteMetadata> execute() { + public HoodieWriteMetadata> execute() { return super.execute(preppedRecords); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertCommitActionExecutor.java index c914384cb21ed..ccee9cf5a7164 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertCommitActionExecutor.java @@ -20,30 +20,29 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; 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.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; - import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.spark.api.java.JavaRDD; public class SparkUpsertCommitActionExecutor> extends BaseSparkCommitActionExecutor { - private JavaRDD> inputRecordsRDD; + private final HoodieData> inputRecordsRDD; public SparkUpsertCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD) { + String instantTime, HoodieData> inputRecordsRDD) { super(context, config, table, instantTime, WriteOperationType.UPSERT); this.inputRecordsRDD = inputRecordsRDD; } @Override - public HoodieWriteMetadata> execute() { - return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, + public HoodieWriteMetadata> execute() { + return HoodieWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(), this, operationType); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertPreppedCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertPreppedCommitActionExecutor.java index e36073fd17d6d..73d408593bd37 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertPreppedCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkUpsertPreppedCommitActionExecutor.java @@ -20,29 +20,28 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; 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.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; - import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.spark.api.java.JavaRDD; public class SparkUpsertPreppedCommitActionExecutor> extends BaseSparkCommitActionExecutor { - private final JavaRDD> preppedRecords; + private final HoodieData> preppedRecords; public SparkUpsertPreppedCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> preppedRecords) { + String instantTime, HoodieData> preppedRecords) { super(context, config, table, instantTime, WriteOperationType.UPSERT_PREPPED); this.preppedRecords = preppedRecords; } @Override - public HoodieWriteMetadata> execute() { + public HoodieWriteMetadata> execute() { return super.execute(preppedRecords); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java index 6ca4408a7bba2..61cb1ffd27bd1 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java @@ -19,7 +19,6 @@ package org.apache.hudi.table.action.compact; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.utils.SparkMemoryUtils; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; @@ -27,10 +26,9 @@ 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.data.HoodieJavaRDD; import org.apache.hudi.table.HoodieTable; -import org.apache.spark.api.java.JavaRDD; +import static org.apache.hudi.config.HoodieWriteConfig.WRITE_STATUS_STORAGE_LEVEL_VALUE; /** * Compacts a hoodie table with merge on read storage. Computes all possible compactions, @@ -39,7 +37,7 @@ */ @SuppressWarnings("checkstyle:LineLength") public class HoodieSparkMergeOnReadTableCompactor - extends HoodieCompactor>, JavaRDD, JavaRDD> { + extends HoodieCompactor>, HoodieData, HoodieData> { @Override public void preCompact( @@ -53,6 +51,6 @@ public void preCompact( @Override public void maybePersist(HoodieData writeStatus, HoodieWriteConfig config) { - HoodieJavaRDD.getJavaRDD(writeStatus).persist(SparkMemoryUtils.getWriteStatusStorageLevel(config.getProps())); + writeStatus.persist(config.getString(WRITE_STATUS_STORAGE_LEVEL_VALUE)); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/BaseSparkDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/BaseSparkDeltaCommitActionExecutor.java index 222506e7bbb36..61e6f25af9429 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/BaseSparkDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/BaseSparkDeltaCommitActionExecutor.java @@ -18,7 +18,6 @@ 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; @@ -42,13 +41,14 @@ import java.util.Collections; import java.util.Iterator; import java.util.List; +import java.util.Map; public abstract class BaseSparkDeltaCommitActionExecutor> extends BaseSparkCommitActionExecutor { private static final Logger LOG = LogManager.getLogger(BaseSparkDeltaCommitActionExecutor.class); // UpsertPartitioner for MergeOnRead table type - private SparkUpsertDeltaCommitPartitioner mergeOnReadUpsertPartitioner; + private SparkUpsertDeltaCommitPartitioner mergeOnReadUpsertPartitioner; public BaseSparkDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, String instantTime, WriteOperationType operationType) { @@ -66,7 +66,7 @@ public Partitioner getUpsertPartitioner(WorkloadProfile profile) { if (profile == null) { throw new HoodieUpsertException("Need workload profile to construct the upsert partitioner."); } - mergeOnReadUpsertPartitioner = new SparkUpsertDeltaCommitPartitioner(profile, (HoodieSparkEngineContext) context, table, config); + mergeOnReadUpsertPartitioner = new SparkUpsertDeltaCommitPartitioner<>(profile, (HoodieSparkEngineContext) context, table, config); return mergeOnReadUpsertPartitioner; } @@ -79,7 +79,7 @@ public Iterator> handleUpdate(String partitionPath, String fil 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, table, + HoodieAppendHandle appendHandle = new HoodieAppendHandle<>(config, instantTime, table, partitionPath, fileId, recordItr, taskContextSupplier); appendHandle.doAppend(); return Collections.singletonList(appendHandle.close()).iterator(); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertDeltaCommitActionExecutor.java index 6f23e41773bbd..190a714e44612 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertDeltaCommitActionExecutor.java @@ -20,6 +20,7 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -31,25 +32,23 @@ import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.commit.SparkBulkInsertHelper; -import org.apache.spark.api.java.JavaRDD; - import java.util.Map; public class SparkBulkInsertDeltaCommitActionExecutor> extends BaseSparkDeltaCommitActionExecutor { - private final JavaRDD> inputRecordsRDD; - private final Option>>> bulkInsertPartitioner; + private final HoodieData> inputRecordsRDD; + private final Option bulkInsertPartitioner; public SparkBulkInsertDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD, - Option>>> bulkInsertPartitioner) { + String instantTime, HoodieData> inputRecordsRDD, + Option bulkInsertPartitioner) { this(context, config, table, instantTime, inputRecordsRDD, bulkInsertPartitioner, Option.empty()); } public SparkBulkInsertDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD, - Option>>> bulkInsertPartitioner, + String instantTime, HoodieData> inputRecordsRDD, + Option bulkInsertPartitioner, Option> extraMetadata) { super(context, config, table, instantTime, WriteOperationType.BULK_INSERT, extraMetadata); this.inputRecordsRDD = inputRecordsRDD; @@ -57,7 +56,7 @@ public SparkBulkInsertDeltaCommitActionExecutor(HoodieSparkEngineContext context } @Override - public HoodieWriteMetadata> execute() { + public HoodieWriteMetadata> execute() { try { return SparkBulkInsertHelper.newInstance().bulkInsert(inputRecordsRDD, instantTime, table, config, this, true, bulkInsertPartitioner); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertPreppedDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertPreppedDeltaCommitActionExecutor.java index be5b903c7642d..c01bce2b9cf35 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertPreppedDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkBulkInsertPreppedDeltaCommitActionExecutor.java @@ -20,6 +20,7 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -31,25 +32,23 @@ import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.commit.SparkBulkInsertHelper; -import org.apache.spark.api.java.JavaRDD; - public class SparkBulkInsertPreppedDeltaCommitActionExecutor> extends BaseSparkDeltaCommitActionExecutor { - private final JavaRDD> preppedInputRecordRdd; - private final Option>>> bulkInsertPartitioner; + private final HoodieData> preppedInputRecordRdd; + private final Option bulkInsertPartitioner; public SparkBulkInsertPreppedDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> preppedInputRecordRdd, - Option>>> bulkInsertPartitioner) { + String instantTime, HoodieData> 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 SparkBulkInsertHelper.newInstance().bulkInsert(preppedInputRecordRdd, instantTime, table, config, this, false, bulkInsertPartitioner); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkDeleteDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkDeleteDeltaCommitActionExecutor.java index 7cff563571459..9a5b08df288d6 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkDeleteDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkDeleteDeltaCommitActionExecutor.java @@ -20,30 +20,29 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; 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.HoodieWriteMetadata; -import org.apache.hudi.table.action.commit.SparkDeleteHelper; - -import org.apache.spark.api.java.JavaRDD; +import org.apache.hudi.table.action.commit.HoodieDeleteHelper; public class SparkDeleteDeltaCommitActionExecutor> extends BaseSparkDeltaCommitActionExecutor { - private final JavaRDD keys; + private final HoodieData keys; public SparkDeleteDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD keys) { + String instantTime, HoodieData keys) { super(context, config, table, instantTime, WriteOperationType.DELETE); this.keys = keys; } @Override - public HoodieWriteMetadata> execute() { - return SparkDeleteHelper.newInstance().execute(instantTime, keys, context, config, table, this); + public HoodieWriteMetadata> execute() { + return HoodieDeleteHelper.newInstance().execute(instantTime, keys, context, config, table, this); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertDeltaCommitActionExecutor.java index 7e38823fc8838..4889460c467fa 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertDeltaCommitActionExecutor.java @@ -20,31 +20,30 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; 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.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.hudi.table.action.commit.SparkWriteHelper; - -import org.apache.spark.api.java.JavaRDD; +import org.apache.hudi.table.action.commit.HoodieWriteHelper; public class SparkInsertDeltaCommitActionExecutor> extends BaseSparkDeltaCommitActionExecutor { - private final JavaRDD> inputRecordsRDD; + private final HoodieData> inputRecordsRDD; public SparkInsertDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD) { + String instantTime, HoodieData> inputRecordsRDD) { super(context, config, table, instantTime, WriteOperationType.INSERT); this.inputRecordsRDD = inputRecordsRDD; } @Override - public HoodieWriteMetadata> execute() { - return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, + public HoodieWriteMetadata> execute() { + return HoodieWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(),this, operationType); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertPreppedDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertPreppedDeltaCommitActionExecutor.java index e401d9555e434..dbf0cbc676118 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertPreppedDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkInsertPreppedDeltaCommitActionExecutor.java @@ -20,6 +20,7 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -27,22 +28,20 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.spark.api.java.JavaRDD; - public class SparkInsertPreppedDeltaCommitActionExecutor> extends BaseSparkDeltaCommitActionExecutor { - private final JavaRDD> preppedRecords; + private final HoodieData> preppedRecords; public SparkInsertPreppedDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> preppedRecords) { + String instantTime, HoodieData> preppedRecords) { super(context, config, table, instantTime, WriteOperationType.INSERT_PREPPED); this.preppedRecords = preppedRecords; } @Override - public HoodieWriteMetadata> execute() { + public HoodieWriteMetadata> execute() { return super.execute(preppedRecords); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitActionExecutor.java index c63be6289004d..67ecb9a8cbc06 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitActionExecutor.java @@ -18,32 +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.data.HoodieData; 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.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.hudi.table.action.commit.SparkWriteHelper; - -import org.apache.spark.api.java.JavaRDD; +import org.apache.hudi.table.action.commit.HoodieWriteHelper; public class SparkUpsertDeltaCommitActionExecutor> extends BaseSparkDeltaCommitActionExecutor { - private JavaRDD> inputRecordsRDD; + private final HoodieData> inputRecordsRDD; public SparkUpsertDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> inputRecordsRDD) { + String instantTime, HoodieData> inputRecordsRDD) { super(context, config, table, instantTime, WriteOperationType.UPSERT); this.inputRecordsRDD = inputRecordsRDD; } @Override - public HoodieWriteMetadata execute() { - return SparkWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, + public HoodieWriteMetadata> execute() { + return HoodieWriteHelper.newInstance().write(instantTime, inputRecordsRDD, context, table, config.shouldCombineBeforeUpsert(), config.getUpsertShuffleParallelism(),this, operationType); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertPreppedDeltaCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertPreppedDeltaCommitActionExecutor.java index f593fea779029..9540030eba157 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertPreppedDeltaCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertPreppedDeltaCommitActionExecutor.java @@ -20,6 +20,7 @@ import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.WriteOperationType; @@ -27,22 +28,20 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.spark.api.java.JavaRDD; - public class SparkUpsertPreppedDeltaCommitActionExecutor> extends BaseSparkDeltaCommitActionExecutor { - private final JavaRDD> preppedRecords; + private final HoodieData> preppedRecords; public SparkUpsertPreppedDeltaCommitActionExecutor(HoodieSparkEngineContext context, HoodieWriteConfig config, HoodieTable table, - String instantTime, JavaRDD> preppedRecords) { + String instantTime, HoodieData> preppedRecords) { super(context, config, table, instantTime, WriteOperationType.UPSERT_PREPPED); this.preppedRecords = preppedRecords; } @Override - public HoodieWriteMetadata> execute() { + public HoodieWriteMetadata> execute() { return super.execute(preppedRecords); } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java index 6e67bd69bdfd7..ce0cc37c78e51 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java @@ -32,6 +32,7 @@ import org.apache.hudi.client.validator.SqlQueryEqualityPreCommitValidator; import org.apache.hudi.client.validator.SqlQuerySingleResultPreCommitValidator; import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.fs.FSUtils; @@ -76,6 +77,7 @@ import org.apache.hudi.config.HoodiePreCommitValidatorConfig; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.exception.HoodieCorruptedDataException; import org.apache.hudi.exception.HoodieIOException; @@ -83,17 +85,19 @@ import org.apache.hudi.exception.HoodieRollbackException; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.exception.HoodieValidationException; +import org.apache.hudi.execution.bulkinsert.RDDCustomColumnsSortPartitioner; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.HoodieIndex.IndexType; import org.apache.hudi.io.HoodieMergeHandle; import org.apache.hudi.keygen.BaseKeyGenerator; import org.apache.hudi.keygen.KeyGenerator; import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory; +import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.table.HoodieSparkCopyOnWriteTable; 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.commit.SparkWriteHelper; +import org.apache.hudi.table.action.commit.HoodieWriteHelper; import org.apache.hudi.table.marker.WriteMarkersFactory; import org.apache.hudi.testutils.HoodieClientTestBase; import org.apache.hudi.testutils.HoodieClientTestUtils; @@ -451,13 +455,13 @@ private void testDeduplication( HoodieRecord recordThree = new HoodieAvroRecord(keyTwo, dataGen.generateRandomValue(keyTwo, newCommitTime)); - JavaRDD> records = - jsc.parallelize(Arrays.asList(recordOne, recordTwo, recordThree), 1); + HoodieData> records = HoodieJavaRDD.of( + jsc.parallelize(Arrays.asList(recordOne, recordTwo, recordThree), 1)); // Global dedup should be done based on recordKey only HoodieIndex index = mock(HoodieIndex.class); when(index.isGlobal()).thenReturn(true); - List> dedupedRecs = SparkWriteHelper.newInstance().deduplicateRecords(records, index, 1).collect(); + List> dedupedRecs = HoodieWriteHelper.newInstance().deduplicateRecords(records, index, 1).collectAsList(); assertEquals(1, dedupedRecs.size()); assertEquals(dedupedRecs.get(0).getPartitionPath(), recordThree.getPartitionPath()); assertNodupesWithinPartition(dedupedRecs); @@ -465,7 +469,7 @@ private void testDeduplication( // non-Global dedup should be done based on both recordKey and partitionPath index = mock(HoodieIndex.class); when(index.isGlobal()).thenReturn(false); - dedupedRecs = SparkWriteHelper.newInstance().deduplicateRecords(records, index, 1).collect(); + dedupedRecs = HoodieWriteHelper.newInstance().deduplicateRecords(records, index, 1).collectAsList(); assertEquals(2, dedupedRecs.size()); assertNodupesWithinPartition(dedupedRecs); @@ -779,6 +783,20 @@ private void testHoodieConcatHandleOnDupInserts(HoodieWriteConfig config, boolea firstInsertRecords + secondInsertRecords, 2, false, config.populateMetaFields()); } + @Test + public void testBulkInsertWithCustomPartitioner() { + HoodieWriteConfig config = getConfigBuilder().withRollbackUsingMarkers(true).build(); + try (SparkRDDWriteClient client = getHoodieWriteClient(config)) { + final String commitTime1 = "001"; + client.startCommitWithTime(commitTime1); + List inserts1 = dataGen.generateInserts(commitTime1, 100); + JavaRDD insertRecordsRDD1 = jsc.parallelize(inserts1, 10); + BulkInsertPartitioner> partitioner = new RDDCustomColumnsSortPartitioner(new String[]{"rider"}, HoodieTestDataGenerator.AVRO_SCHEMA, false); + List statuses = client.bulkInsert(insertRecordsRDD1, commitTime1, Option.of(partitioner)).collect(); + assertNoWriteErrors(statuses); + } + } + /** * Tests deletion of records. */ @@ -2594,7 +2612,7 @@ private HoodieWriteConfig getParallelWritingWriteConfig(HoodieFailedWritesCleani .withProperties(populateMetaFields ? new Properties() : getPropertiesForKeyGen()).build(); } - public static class FailingPreCommitValidator> extends SparkPreCommitValidator { + public static class FailingPreCommitValidator> extends SparkPreCommitValidator { public FailingPreCommitValidator(HoodieSparkTable table, HoodieEngineContext context, HoodieWriteConfig config) { super(table, context, config); diff --git a/hudi-client/hudi-spark-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 index dcc41addc8f31..b9f025223b7df 100644 --- a/hudi-client/hudi-spark-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 @@ -39,6 +39,7 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieClusteringConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.HoodieIndex.IndexType; import org.apache.hudi.metadata.HoodieTableMetadataWriter; @@ -556,7 +557,7 @@ public void testHandleUpdateWithMultiplePartitions() throws Exception { // initialize partitioner hoodieTable.getHoodieView().sync(); BaseSparkDeltaCommitActionExecutor actionExecutor = new SparkDeleteDeltaCommitActionExecutor(context(), cfg, hoodieTable, - newDeleteTime, deleteRDD); + newDeleteTime, HoodieJavaRDD.of(deleteRDD)); actionExecutor.getUpsertPartitioner(new WorkloadProfile(buildProfile(deleteRDD))); final List> deleteStatus = jsc().parallelize(Arrays.asList(1)).map(x -> { return actionExecutor.handleUpdate(partitionPath, fileId, fewRecordsForDelete.iterator()); diff --git a/hudi-client/hudi-spark-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 index 53cd6e5d1e749..0b29cf25f9e3f 100644 --- a/hudi-client/hudi-spark-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 @@ -21,6 +21,7 @@ import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.bloom.BloomFilter; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieAvroRecord; import org.apache.hudi.common.model.HoodieKey; @@ -38,6 +39,7 @@ import org.apache.hudi.config.HoodieLayoutConfig; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.data.HoodieJavaRDD; import org.apache.hudi.hadoop.HoodieParquetInputFormat; import org.apache.hudi.hadoop.utils.HoodieHiveUtils; import org.apache.hudi.index.HoodieIndex; @@ -325,7 +327,7 @@ public void testMetadataAggregateFromWriteStatus() throws Exception { // Insert new records BaseSparkCommitActionExecutor actionExecutor = new SparkInsertCommitActionExecutor(context, config, table, - firstCommitTime, jsc.parallelize(records)); + firstCommitTime, context.parallelize(records)); List writeStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> { return actionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), records.iterator()); }).flatMap(Transformations::flattenAsIterator).collect(); @@ -368,7 +370,7 @@ public void testInsertRecords() throws Exception { // Insert new records final List recs2 = records; BaseSparkCommitActionExecutor actionExecutor = new SparkInsertPreppedCommitActionExecutor(context, config, table, - instantTime, jsc.parallelize(recs2)); + instantTime, context.parallelize(recs2)); List returnedStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> { return actionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), recs2.iterator()); }).flatMap(Transformations::flattenAsIterator).collect(); @@ -389,7 +391,7 @@ public void testInsertRecords() throws Exception { // Insert new records final List recs3 = records; BaseSparkCommitActionExecutor newActionExecutor = new SparkUpsertPreppedCommitActionExecutor(context, config, table, - instantTime, jsc.parallelize(recs3)); + instantTime, context.parallelize(recs3)); returnedStatuses = jsc.parallelize(Arrays.asList(1)).map(x -> { return newActionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), recs3.iterator()); }).flatMap(Transformations::flattenAsIterator).collect(); @@ -422,7 +424,7 @@ public void testFileSizeUpsertRecords() throws Exception { // Insert new records BaseSparkCommitActionExecutor actionExecutor = new SparkUpsertCommitActionExecutor(context, config, table, - instantTime, jsc.parallelize(records)); + instantTime, context.parallelize(records)); jsc.parallelize(Arrays.asList(1)) .map(i -> actionExecutor.handleInsert(FSUtils.createNewFileIdPfx(), records.iterator())) .map(Transformations::flatten).collect(); @@ -452,7 +454,7 @@ public void testInsertUpsertWithHoodieAvroPayload() throws Exception { // Perform inserts of 100 records to test CreateHandle and BufferedExecutor final List inserts = dataGen.generateInsertsWithHoodieAvroPayload(instantTime, 100); BaseSparkCommitActionExecutor actionExecutor = new SparkInsertCommitActionExecutor(context, config, table, - instantTime, jsc.parallelize(inserts)); + instantTime, context.parallelize(inserts)); final List> ws = jsc.parallelize(Arrays.asList(1)).map(x -> { return actionExecutor.handleInsert(UUID.randomUUID().toString(), inserts.iterator()); }).map(Transformations::flatten).collect(); @@ -466,7 +468,7 @@ public void testInsertUpsertWithHoodieAvroPayload() throws Exception { long numRecordsInPartition = updates.stream().filter(u -> u.getPartitionPath().equals(partitionPath)).count(); table = (HoodieSparkCopyOnWriteTable) HoodieSparkTable.create(config, context, HoodieTableMetaClient.reload(metaClient)); BaseSparkCommitActionExecutor newActionExecutor = new SparkUpsertCommitActionExecutor(context, config, table, - instantTime, jsc.parallelize(updates)); + instantTime, context.parallelize(updates)); final List> updateStatus = jsc.parallelize(Arrays.asList(1)).map(x -> { return newActionExecutor.handleUpdate(partitionPath, fileId, updates.iterator()); }).map(Transformations::flatten).collect(); @@ -486,8 +488,8 @@ public void testBulkInsertRecords(String bulkInsertMode) throws Exception { // Insert new records final JavaRDD inputRecords = generateTestRecordsForBulkInsert(jsc); SparkBulkInsertCommitActionExecutor bulkInsertExecutor = new SparkBulkInsertCommitActionExecutor( - context, config, table, instantTime, inputRecords, Option.empty()); - List returnedStatuses = ((JavaRDD)bulkInsertExecutor.execute().getWriteStatuses()).collect(); + context, config, table, instantTime, HoodieJavaRDD.of(inputRecords), Option.empty()); + List returnedStatuses = ((HoodieData) bulkInsertExecutor.execute().getWriteStatuses()).collectAsList(); verifyStatusResult(returnedStatuses, generateExpectedPartitionNumRecords(inputRecords)); } diff --git a/hudi-client/hudi-spark-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 deleted file mode 100644 index 2d852f8107ef0..0000000000000 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestDeleteHelper.java +++ /dev/null @@ -1,194 +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.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.data.HoodieJavaRDD; -import org.apache.hudi.index.HoodieIndex; -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 java.util.Collections; -import java.util.List; - -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyInt; -import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.doNothing; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -@ExtendWith(MockitoExtension.class) -public class TestDeleteHelper { - - private enum CombineTestMode { - None, GlobalIndex, NoneGlobalIndex; - } - - private static final String BASE_PATH = "/tmp/"; - private static final boolean WITH_COMBINE = true; - private static final boolean WITHOUT_COMBINE = false; - private static final int DELETE_PARALLELISM = 200; - - @Mock - private HoodieIndex 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; - - @BeforeEach - public void setUp() { - when(table.getIndex()).thenReturn(index); - when(context.getJavaSparkContext()).thenReturn(jsc); - } - - @Test - public void deleteWithEmptyRDDShouldNotExecute() { - rddToDelete = mockEmptyHoodieKeyRdd(); - config = newWriteConfig(WITHOUT_COMBINE); - - SparkDeleteHelper.newInstance().execute("test-time", rddToDelete, context, config, table, executor); - - verify(rddToDelete, never()).repartition(DELETE_PARALLELISM); - verifyNoDeleteExecution(); - } - - @Test - public void deleteWithoutCombineShouldRepartitionForNonEmptyRdd() { - rddToDelete = newHoodieKeysRddMock(2, CombineTestMode.None); - config = newWriteConfig(WITHOUT_COMBINE); - - SparkDeleteHelper.newInstance().execute("test-time", rddToDelete, context, config, table, executor); - - verify(rddToDelete, times(1)).repartition(DELETE_PARALLELISM); - verifyDeleteExecution(); - } - - @Test - public void deleteWithCombineShouldRepartitionForNonEmptyRddAndNonGlobalIndex() { - rddToDelete = newHoodieKeysRddMock(2, CombineTestMode.NoneGlobalIndex); - config = newWriteConfig(WITH_COMBINE); - - SparkDeleteHelper.newInstance().execute("test-time", rddToDelete, context, config, table, executor); - - verify(rddToDelete, times(1)).distinct(DELETE_PARALLELISM); - verifyDeleteExecution(); - } - - @Test - public void deleteWithCombineShouldRepartitionForNonEmptyRddAndGlobalIndex() { - rddToDelete = newHoodieKeysRddMock(2, CombineTestMode.GlobalIndex); - config = newWriteConfig(WITH_COMBINE); - when(index.isGlobal()).thenReturn(true); - - SparkDeleteHelper.newInstance().execute("test-time", rddToDelete, context, config, table, executor); - - verify(keyPairs, times(1)).reduceByKey(any(), eq(DELETE_PARALLELISM)); - verifyDeleteExecution(); - } - - private void verifyDeleteExecution() { - verify(executor, times(1)).execute(any()); - verify(metadata, times(1)).setIndexLookupDuration(any()); - } - - private void verifyNoDeleteExecution() { - verify(executor, never()).execute(any()); - } - - private HoodieWriteConfig newWriteConfig(boolean combine) { - return HoodieWriteConfig.newBuilder() - .combineDeleteInput(combine) - .withPath(BASE_PATH) - .withDeleteParallelism(DELETE_PARALLELISM) - .build(); - } - - private JavaRDD newHoodieKeysRddMock(int howMany, CombineTestMode combineMode) { - JavaRDD keysToDelete = mock(JavaRDD.class); - - JavaRDD recordsRdd = mock(JavaRDD.class); - when(recordsRdd.filter(any())).thenReturn(recordsRdd); - when(recordsRdd.isEmpty()).thenReturn(howMany <= 0); - when(index.tagLocation(any(), any(), any())).thenReturn(HoodieJavaRDD.of(recordsRdd)); - - if (combineMode == CombineTestMode.GlobalIndex) { - when(keyPairs.reduceByKey(any(), anyInt())).thenReturn(keyPairs); - when(keyPairs.values()).thenReturn(keysToDelete); - when(keysToDelete.keyBy(any())).thenReturn(keyPairs); - } else if (combineMode == CombineTestMode.NoneGlobalIndex) { - when(keysToDelete.distinct(anyInt())).thenReturn(keysToDelete); - } else if (combineMode == CombineTestMode.None) { - List parts = mock(List.class); - when(parts.isEmpty()).thenReturn(howMany <= 0); - when(keysToDelete.repartition(anyInt())).thenReturn(keysToDelete); - when(keysToDelete.partitions()).thenReturn(parts); - } - - when(keysToDelete.map(any())).thenReturn(recordsRdd); - when(executor.execute(any())).thenReturn(metadata); - return keysToDelete; - } - - private JavaRDD mockEmptyHoodieKeyRdd() { - JavaRDD emptyRdd = mock(JavaRDD.class); - doReturn(true).when(emptyRdd).isEmpty(); - doReturn(Collections.emptyList()).when(emptyRdd).partitions(); - doReturn(emptyRdd).when(emptyRdd).map(any()); - - doReturn(HoodieJavaRDD.of(emptyRdd)).when(index).tagLocation(any(), any(), any()); - doReturn(emptyRdd).when(emptyRdd).filter(any()); - - doNothing().when(executor).saveWorkloadProfileMetadataToInflight(any(), anyString()); - doReturn(emptyRdd).when(jsc).emptyRDD(); - return emptyRdd; - } - -} diff --git a/hudi-client/hudi-spark-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 index 9afe5f3533cac..59174a9371a58 100644 --- a/hudi-client/hudi-spark-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 @@ -21,6 +21,7 @@ import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieRecord; @@ -195,12 +196,12 @@ public void testWriteStatusContentsAfterCompaction() throws Exception { String compactionInstantTime = "102"; table.scheduleCompaction(context, compactionInstantTime, Option.empty()); table.getMetaClient().reloadActiveTimeline(); - JavaRDD result = (JavaRDD) table.compact( + HoodieData result = (HoodieData) table.compact( context, compactionInstantTime).getWriteStatuses(); // Verify that all partition paths are present in the WriteStatus result for (String partitionPath : dataGen.getPartitionPaths()) { - List writeStatuses = result.collect(); + List writeStatuses = result.collectAsList(); assertTrue(writeStatuses.stream() .filter(writeStatus -> writeStatus.getStat().getPartitionPath().contentEquals(partitionPath)).count() > 0); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieData.java b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieData.java index 093fd439db09c..4e8d2b7eceaee 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieData.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieData.java @@ -97,6 +97,12 @@ public abstract HoodieData mapPartitions( */ public abstract HoodieData distinct(); + public abstract HoodieData distinct(int parallelism); + + public abstract HoodieData distinctWithKey(SerializableFunction keyGetter, int parallelism); + + public abstract HoodieData filter(SerializableFunction filterFunc); + /** * Unions this {@link HoodieData} with other {@link HoodieData}. * @param other {@link HoodieData} of interest. @@ -108,4 +114,6 @@ public abstract HoodieData mapPartitions( * @return collected results in {@link List}. */ public abstract List collectAsList(); + + public abstract HoodieData repartition(int parallelism); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieList.java b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieList.java index 94416192abfb9..c23e712cf41ae 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieList.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieList.java @@ -132,6 +132,26 @@ public HoodieData distinct() { return HoodieList.of(new ArrayList<>(new HashSet<>(listData))); } + @Override + public HoodieData distinct(int parallelism) { + return distinct(); + } + + @Override + public HoodieData distinctWithKey(SerializableFunction keyGetter, int parallelism) { + return mapToPair(i -> Pair.of(keyGetter.apply(i), i)) + .reduceByKey((value1, value2) -> value1, parallelism) + .values(); + } + + @Override + public HoodieData filter(SerializableFunction filterFunc) { + return HoodieList.of(listData + .stream() + .filter(i -> throwingMapWrapper(filterFunc).apply(i)) + .collect(Collectors.toList())); + } + @Override public HoodieData union(HoodieData other) { List unionResult = new ArrayList<>(); @@ -144,4 +164,10 @@ public HoodieData union(HoodieData other) { public List collectAsList() { return listData; } + + @Override + public HoodieData repartition(int parallelism) { + // no op + return this; + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieMapPair.java b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieMapPair.java index c941231e617f1..1e125c90a190b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieMapPair.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodieMapPair.java @@ -20,6 +20,7 @@ package org.apache.hudi.common.data; import org.apache.hudi.common.function.FunctionWrapper; +import org.apache.hudi.common.function.SerializableBiFunction; import org.apache.hudi.common.function.SerializableFunction; import org.apache.hudi.common.function.SerializablePairFunction; import org.apache.hudi.common.util.Option; @@ -27,6 +28,7 @@ import org.apache.hudi.common.util.collection.Pair; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -110,6 +112,15 @@ public Map countByKey() { Collectors.toMap(Map.Entry::getKey, entry -> (long) entry.getValue().size())); } + @Override + public HoodiePairData reduceByKey(SerializableBiFunction func, int parallelism) { + return HoodieMapPair.of(mapPairData.entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> { + Option reducedValue = Option.fromJavaOptional(e.getValue().stream().reduce(func::apply)); + return reducedValue.isPresent() ? Collections.singletonList(reducedValue.get()) : Collections.emptyList(); + }))); + } + @Override public HoodieData map(SerializableFunction, O> func) { Function, O> throwableFunc = throwingMapWrapper(func); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodiePairData.java b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodiePairData.java index b9bdcb3d9ff4a..9ff52793d6f0e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/data/HoodiePairData.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/data/HoodiePairData.java @@ -19,6 +19,7 @@ package org.apache.hudi.common.data; +import org.apache.hudi.common.function.SerializableBiFunction; import org.apache.hudi.common.function.SerializableFunction; import org.apache.hudi.common.function.SerializablePairFunction; import org.apache.hudi.common.util.Option; @@ -72,6 +73,8 @@ public abstract class HoodiePairData implements Serializable { */ public abstract Map countByKey(); + public abstract HoodiePairData reduceByKey(SerializableBiFunction func, int parallelism); + /** * @param func serializable map function. * @param output object type. diff --git a/hudi-common/src/test/java/org/apache/hudi/common/data/TestHoodieList.java b/hudi-common/src/test/java/org/apache/hudi/common/data/TestHoodieList.java new file mode 100644 index 0000000000000..6130d4af1094e --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/common/data/TestHoodieList.java @@ -0,0 +1,50 @@ +/* + * 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.common.data; + +import org.apache.hudi.common.util.collection.Pair; + +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.util.Arrays; +import java.util.List; +import java.util.stream.Stream; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +class TestHoodieList { + + private static Stream distinctWithKey() { + return Stream.of( + Arguments.of( + Arrays.asList(Pair.of("k1", 1), Pair.of("k2", 2)), + Arrays.asList(Pair.of("k1", 1), Pair.of("k1", 10), Pair.of("k1", 100), Pair.of("k2", 2))) + ); + } + + @ParameterizedTest + @MethodSource + void distinctWithKey(List> expected, List> originalList) { + List> distinctList = HoodieList.of(originalList).distinctWithKey(Pair::getLeft, 1).collectAsList(); + assertEquals(expected, distinctList); + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/common/data/TestHoodieMapPair.java b/hudi-common/src/test/java/org/apache/hudi/common/data/TestHoodieMapPair.java index 86b1a213ba639..20e9a8f5d9b13 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/data/TestHoodieMapPair.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/data/TestHoodieMapPair.java @@ -25,6 +25,9 @@ import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; import java.util.ArrayList; import java.util.Arrays; @@ -33,7 +36,10 @@ import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import java.util.stream.Stream; +import static org.apache.hudi.common.util.CollectionUtils.createImmutableList; +import static org.apache.hudi.common.util.CollectionUtils.createImmutableMap; import static org.junit.jupiter.api.Assertions.assertEquals; public class TestHoodieMapPair { @@ -117,6 +123,29 @@ public void testMapToPair() { }))); } + private static Stream testReduceByKey() { + return Stream.of( + Arguments.of( + createImmutableMap( + Pair.of(1, createImmutableList(1001)), + Pair.of(2, createImmutableList(2001)), + Pair.of(3, createImmutableList(3001)), + Pair.of(4, createImmutableList())), + createImmutableMap( + Pair.of(1, createImmutableList(1001, 1002, 1003)), + Pair.of(2, createImmutableList(2001, 2002)), + Pair.of(3, createImmutableList(3001)), + Pair.of(4, createImmutableList()))) + ); + } + + @ParameterizedTest + @MethodSource + public void testReduceByKey(Map> expected, Map> original) { + HoodiePairData reduced = HoodieMapPair.of(original).reduceByKey((a, b) -> a, 1); + assertEquals(expected, HoodieMapPair.getMapPair(reduced)); + } + @Test public void testLeftOuterJoinSingleValuePerKey() { HoodiePairData pairData1 = constructTestMapPairData(Arrays.asList(