Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -246,7 +246,21 @@ public List<WriteStatus> bulkInsert(List<HoodieRecord<T>> records, String instan

@Override
public List<WriteStatus> bulkInsertPreppedRecords(List<HoodieRecord<T>> preppedRecords, String instantTime, Option<BulkInsertPartitioner> bulkInsertPartitioner) {
throw new HoodieNotSupportedException("BulkInsertPrepped operation is not supported yet");
// only used for metadata table, the bulk_insert happens in single JVM process
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> table =
initTable(WriteOperationType.BULK_INSERT_PREPPED, Option.ofNullable(instantTime));
table.validateInsertSchema();
preWrite(instantTime, WriteOperationType.BULK_INSERT_PREPPED, table.getMetaClient());
Map<String, List<HoodieRecord<T>>> preppedRecordsByFileId = preppedRecords.stream().parallel()
.collect(Collectors.groupingBy(r -> r.getCurrentLocation().getFileId()));
return preppedRecordsByFileId.values().stream().parallel().map(records -> {
HoodieWriteMetadata<List<WriteStatus>> result;
records.get(0).getCurrentLocation().setInstantTime("I");
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why setting instantTime to I? Should it not be instantTime passed to the method?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Flink relies on this flag to distinguish which write handle to use. A little hacky but it's the minimum change right now.

try (AutoCloseableWriteHandle closeableHandle = new AutoCloseableWriteHandle(records, instantTime, table, true)) {
result = ((HoodieFlinkTable<T>) table).bulkInsertPrepped(context, closeableHandle.getWriteHandle(), instantTime, records);
}
return postWrite(result, instantTime, table);
}).flatMap(Collection::stream).collect(Collectors.toList());
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@
import org.slf4j.LoggerFactory;

import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;

Expand Down Expand Up @@ -107,6 +108,17 @@ protected void initRegistry() {

@Override
protected void commit(String instantTime, Map<MetadataPartitionType, HoodieData<HoodieRecord>> partitionRecordsMap) {
doCommit(instantTime, partitionRecordsMap, false);
}

@Override
protected void bulkCommit(String instantTime, MetadataPartitionType partitionType, HoodieData<HoodieRecord> records, int fileGroupCount) {
Map<MetadataPartitionType, HoodieData<HoodieRecord>> partitionRecordsMap = new HashMap<>();
partitionRecordsMap.put(partitionType, records);
doCommit(instantTime, partitionRecordsMap, true);
}

private void doCommit(String instantTime, Map<MetadataPartitionType, HoodieData<HoodieRecord>> partitionRecordsMap, boolean isInitializing) {
ValidationUtils.checkState(metadataMetaClient != null, "Metadata table is not fully initialized yet.");
HoodieData<HoodieRecord> preppedRecords = prepRecords(partitionRecordsMap);
List<HoodieRecord> preppedRecordList = preppedRecords.collectAsList();
Expand Down Expand Up @@ -149,9 +161,9 @@ protected void commit(String instantTime, Map<MetadataPartitionType, HoodieData<
writeClient.getHeartbeatClient().start(instantTime);
}

List<WriteStatus> statuses = preppedRecordList.size() > 0
? writeClient.upsertPreppedRecords(preppedRecordList, instantTime)
: Collections.emptyList();
List<WriteStatus> statuses = isInitializing
? writeClient.bulkInsertPreppedRecords(preppedRecordList, instantTime, Option.empty())
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

major reason to use bulkInsert is that, we use a custom partitioner based on file group and so the spark tasks will be such that, each spark task will get records pertaining to one file group of interest.

we can try to incorporate that as well. esply with RLI, record mapping to file groups is based on hash. So, we can't have diff set of records routed to one spark task.

Copy link
Contributor Author

@danny0405 danny0405 Jun 10, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Flink does not support flexible partitioner like Spark do, but we can still get some benefits because writing Hfiles directly for initilization of MDT is more efficient. For example, when user enable the MDT for a existing table and there are plenty of metadata records to bootstrap with.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

records to file group mapping is deterministic and we can have only one file written per file group. for eg, if we instanttiate col stats with 4 file groups, we should spin up 4 spark tasks and each spark task should get records pertaining to the file group of interest (remember records are mapped to file group based on hashing). So, if one spark task gets records for all file groups, then we might end up w/ n*m files (where n is no of spark tasks and m is number of file groups) which may not work. we need only m files created and m spark tasks should spin up where each spark tasks writes to just 1 file group.
hope that makes sense.

Copy link
Contributor Author

@danny0405 danny0405 Jun 11, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What the spark partitioner does is to repartition the records by the file group (index) to avoid concurrent write from different tasks into one file group. Flink already does that, even though it is parallelized in single JVM process.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if its already taken care of, then we are good

: writeClient.upsertPreppedRecords(preppedRecordList, instantTime);
// flink does not support auto-commit yet, also the auto commit logic is not complete as BaseHoodieWriteClient now.
writeClient.commit(instantTime, statuses, Option.empty(), HoodieActiveTimeline.DELTA_COMMIT_ACTION, Collections.emptyMap());

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@ public interface ExplicitWriteHandleTable<T> {
/**
* Upsert a batch of new records into Hoodie table at the supplied instantTime.
*
* <p>Specifies the write handle explicitly in order to have fine grained control with
* <p>Specifies the write handle explicitly in order to have fine-grained control with
* the underneath file.
*
* @param context HoodieEngineContext
Expand All @@ -53,7 +53,7 @@ HoodieWriteMetadata<List<WriteStatus>> upsert(
/**
* Insert a batch of new records into Hoodie table at the supplied instantTime.
*
* <p>Specifies the write handle explicitly in order to have fine grained control with
* <p>Specifies the write handle explicitly in order to have fine-grained control with
* the underneath file.
*
* @param context HoodieEngineContext
Expand All @@ -72,7 +72,7 @@ HoodieWriteMetadata<List<WriteStatus>> insert(
* Deletes a list of {@link HoodieKey}s from the Hoodie table, at the supplied instantTime {@link HoodieKey}s will be
* de-duped and non existent keys will be removed before deleting.
*
* <p>Specifies the write handle explicitly in order to have fine grained control with
* <p>Specifies the write handle explicitly in order to have fine-grained control with
* the underneath file.
*
* @param context HoodieEngineContext
Expand All @@ -92,12 +92,12 @@ HoodieWriteMetadata<List<WriteStatus>> delete(
*
* <p>This implementation requires that the input records are already tagged, and de-duped if needed.
*
* <p>Specifies the write handle explicitly in order to have fine grained control with
* <p>Specifies the write handle explicitly in order to have fine-grained control with
* the underneath file.
*
* @param context HoodieEngineContext
* @param instantTime Instant Time for the action
* @param preppedRecords hoodieRecords to upsert
* @param preppedRecords HoodieRecords to upsert
* @return HoodieWriteMetadata
*/
HoodieWriteMetadata<List<WriteStatus>> upsertPrepped(
Expand All @@ -111,12 +111,12 @@ HoodieWriteMetadata<List<WriteStatus>> upsertPrepped(
*
* <p>This implementation requires that the input records are already tagged, and de-duped if needed.
*
* <p>Specifies the write handle explicitly in order to have fine grained control with
* <p>Specifies the write handle explicitly in order to have fine-grained control with
* the underneath file.
*
* @param context HoodieEngineContext
* @param instantTime Instant Time for the action
* @param preppedRecords hoodieRecords to upsert
* @param preppedRecords Hoodie records to insert
* @return HoodieWriteMetadata
*/
HoodieWriteMetadata<List<WriteStatus>> insertPrepped(
Expand All @@ -125,6 +125,25 @@ HoodieWriteMetadata<List<WriteStatus>> insertPrepped(
String instantTime,
List<HoodieRecord<T>> preppedRecords);

/**
* Bulk inserts the given prepared records into the Hoodie table, at the supplied instantTime.
*
* <p>This implementation requires that the input records are already tagged, and de-duped if needed.
*
* <p>Specifies the write handle explicitly in order to have fine-grained control with
* the underneath file.
*
* @param context HoodieEngineContext
* @param instantTime Instant Time for the action
* @param preppedRecords Hoodie records to bulk_insert
* @return HoodieWriteMetadata
*/
HoodieWriteMetadata<List<WriteStatus>> bulkInsertPrepped(
HoodieEngineContext context,
HoodieWriteHandle<?, ?, ?, ?> writeHandle,
String instantTime,
List<HoodieRecord<T>> preppedRecords);

/**
* Replaces all the existing records and inserts the specified new records into Hoodie table at the supplied instantTime,
* for the partition paths contained in input records.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,7 @@
import org.apache.hudi.table.action.clean.CleanActionExecutor;
import org.apache.hudi.table.action.clean.CleanPlanActionExecutor;
import org.apache.hudi.table.action.cluster.ClusteringPlanActionExecutor;
import org.apache.hudi.table.action.commit.FlinkBulkInsertPreppedCommitActionExecutor;
import org.apache.hudi.table.action.commit.FlinkDeleteCommitActionExecutor;
import org.apache.hudi.table.action.commit.FlinkDeletePartitionCommitActionExecutor;
import org.apache.hudi.table.action.commit.FlinkInsertCommitActionExecutor;
Expand Down Expand Up @@ -94,7 +95,7 @@ public HoodieFlinkCopyOnWriteTable(HoodieWriteConfig config, HoodieEngineContext
/**
* Upsert a batch of new records into Hoodie table at the supplied instantTime.
*
* <p>Specifies the write handle explicitly in order to have fine grained control with
* <p>Specifies the write handle explicitly in order to have fine-grained control with
* the underneath file.
*
* @param context HoodieEngineContext
Expand All @@ -114,7 +115,7 @@ public HoodieWriteMetadata<List<WriteStatus>> upsert(
/**
* Insert a batch of new records into Hoodie table at the supplied instantTime.
*
* <p>Specifies the write handle explicitly in order to have fine grained control with
* <p>Specifies the write handle explicitly in order to have fine-grained control with
* the underneath file.
*
* @param context HoodieEngineContext
Expand All @@ -135,7 +136,7 @@ public HoodieWriteMetadata<List<WriteStatus>> insert(
* Deletes a list of {@link HoodieKey}s from the Hoodie table, at the supplied instantTime {@link HoodieKey}s will be
* de-duped and non existent keys will be removed before deleting.
*
* <p>Specifies the write handle explicitly in order to have fine grained control with
* <p>Specifies the write handle explicitly in order to have fine-grained control with
* the underneath file.
*
* @param context HoodieEngineContext
Expand All @@ -157,12 +158,12 @@ public HoodieWriteMetadata<List<WriteStatus>> delete(
*
* <p>This implementation requires that the input records are already tagged, and de-duped if needed.
*
* <p>Specifies the write handle explicitly in order to have fine grained control with
* <p>Specifies the write handle explicitly in order to have fine-grained control with
* the underneath file.
*
* @param context HoodieEngineContext
* @param instantTime Instant Time for the action
* @param preppedRecords hoodieRecords to upsert
* @param preppedRecords Hoodie records to upsert
* @return HoodieWriteMetadata
*/
public HoodieWriteMetadata<List<WriteStatus>> upsertPrepped(
Expand All @@ -178,12 +179,12 @@ public HoodieWriteMetadata<List<WriteStatus>> upsertPrepped(
*
* <p>This implementation requires that the input records are already tagged, and de-duped if needed.
*
* <p>Specifies the write handle explicitly in order to have fine grained control with
* <p>Specifies the write handle explicitly in order to have fine-grained control with
* the underneath file.
*
* @param context HoodieEngineContext
* @param instantTime Instant Time for the action
* @param preppedRecords hoodieRecords to upsert
* @param preppedRecords Hoodie records to insert
* @return HoodieWriteMetadata
*/
public HoodieWriteMetadata<List<WriteStatus>> insertPrepped(
Expand All @@ -194,6 +195,27 @@ public HoodieWriteMetadata<List<WriteStatus>> insertPrepped(
return new FlinkInsertPreppedCommitActionExecutor<>(context, writeHandle, config, this, instantTime, preppedRecords).execute();
}

/**
* Bulk inserts the given prepared records into the Hoodie table, at the supplied instantTime.
*
* <p>This implementation requires that the input records are already tagged, and de-duped if needed.
*
* <p>Specifies the write handle explicitly in order to have fine-grained control with
* the underneath file.
*
* @param context HoodieEngineContext
* @param instantTime Instant Time for the action
* @param preppedRecords Hoodie records to bulk_insert
* @return HoodieWriteMetadata
*/
public HoodieWriteMetadata<List<WriteStatus>> bulkInsertPrepped(
HoodieEngineContext context,
HoodieWriteHandle<?, ?, ?, ?> writeHandle,
String instantTime,
List<HoodieRecord<T>> preppedRecords) {
return new FlinkBulkInsertPreppedCommitActionExecutor<>(context, writeHandle, config, this, instantTime, preppedRecords).execute();
}

@Override
public HoodieWriteMetadata<List<WriteStatus>> insertOverwrite(
HoodieEngineContext context,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,51 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.hudi.table.action.commit;

import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.io.HoodieWriteHandle;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.action.HoodieWriteMetadata;

import java.util.List;

/**
* Flink insert prepped commit action executor.
*/
public class FlinkBulkInsertPreppedCommitActionExecutor<T> extends BaseFlinkCommitActionExecutor<T> {

private final List<HoodieRecord<T>> preppedRecords;

public FlinkBulkInsertPreppedCommitActionExecutor(HoodieEngineContext context,
HoodieWriteHandle<?, ?, ?, ?> writeHandle,
HoodieWriteConfig config, HoodieTable table,
String instantTime, List<HoodieRecord<T>> preppedRecords) {
super(context, writeHandle, config, table, instantTime, WriteOperationType.BULK_INSERT_PREPPED);
this.preppedRecords = preppedRecords;
}

@Override
public HoodieWriteMetadata<List<WriteStatus>> execute() {
return super.execute(preppedRecords);
}
}