[HUDI-1632] Supports merge on read write mode for Flink writer (#2593)
Also supports async compaction with pluggable strategies.
This commit is contained in:
@@ -51,8 +51,8 @@ public class FlinkOptions {
|
||||
.key("path")
|
||||
.stringType()
|
||||
.noDefaultValue()
|
||||
.withDescription("Base path for the target hoodie table."
|
||||
+ "\nThe path would be created if it does not exist,\n"
|
||||
.withDescription("Base path for the target hoodie table.\n"
|
||||
+ "The path would be created if it does not exist,\n"
|
||||
+ "otherwise a Hoodie table expects to be initialized successfully");
|
||||
|
||||
// ------------------------------------------------------------------------
|
||||
@@ -165,6 +165,42 @@ public class FlinkOptions {
|
||||
.defaultValue(128D) // 128MB
|
||||
.withDescription("Batch buffer size in MB to flush data into the underneath filesystem");
|
||||
|
||||
// ------------------------------------------------------------------------
|
||||
// Compaction Options
|
||||
// ------------------------------------------------------------------------
|
||||
|
||||
public static final ConfigOption<Boolean> COMPACTION_ASYNC_ENABLED = ConfigOptions
|
||||
.key("compaction.async.enabled")
|
||||
.booleanType()
|
||||
.defaultValue(true) // default true for MOR write
|
||||
.withDescription("Async Compaction, enabled by default for MOR");
|
||||
|
||||
public static final String NUM_COMMITS = "num_commits";
|
||||
public static final String TIME_ELAPSED = "time_elapsed";
|
||||
public static final String NUM_AND_TIME = "num_and_time";
|
||||
public static final String NUM_OR_TIME = "num_or_time";
|
||||
public static final ConfigOption<String> COMPACTION_TRIGGER_STRATEGY = ConfigOptions
|
||||
.key("compaction.trigger.strategy")
|
||||
.stringType()
|
||||
.defaultValue(NUM_COMMITS) // default true for MOR write
|
||||
.withDescription("Strategy to trigger compaction, options are 'num_commits': trigger compaction when reach N delta commits;\n"
|
||||
+ "'time_elapsed': trigger compaction when time elapsed > N seconds since last compaction;\n"
|
||||
+ "'num_and_time': trigger compaction when both NUM_COMMITS and TIME_ELAPSED are satisfied;\n"
|
||||
+ "'num_or_time': trigger compaction when NUM_COMMITS or TIME_ELAPSED is satisfied.\n"
|
||||
+ "Default is 'num_commits'");
|
||||
|
||||
public static final ConfigOption<Integer> COMPACTION_DELTA_COMMITS = ConfigOptions
|
||||
.key("compaction.delta_commits")
|
||||
.intType()
|
||||
.defaultValue(5)
|
||||
.withDescription("Max delta commits needed to trigger compaction, default 5 commits");
|
||||
|
||||
public static final ConfigOption<Integer> COMPACTION_DELTA_SECONDS = ConfigOptions
|
||||
.key("compaction.delta_seconds")
|
||||
.intType()
|
||||
.defaultValue(3600) // default 1 hour
|
||||
.withDescription("Max delta seconds time needed to trigger compaction, default 1 hour");
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Utilities
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
@@ -40,7 +40,6 @@ import org.apache.flink.runtime.state.FunctionSnapshotContext;
|
||||
import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
|
||||
import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
|
||||
import org.apache.flink.util.Collector;
|
||||
import org.apache.flink.util.Preconditions;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@@ -333,8 +332,11 @@ public class StreamWriteFunction<K, I, O>
|
||||
@SuppressWarnings("unchecked, rawtypes")
|
||||
private void flushBuffer(boolean isFinalBatch) {
|
||||
this.currentInstant = this.writeClient.getInflightAndRequestedInstant(this.config.get(FlinkOptions.TABLE_TYPE));
|
||||
Preconditions.checkNotNull(this.currentInstant,
|
||||
"No inflight instant when flushing data");
|
||||
if (this.currentInstant == null) {
|
||||
// in case there are empty checkpoints that has no input data
|
||||
LOG.info("No inflight instant when flushing data, cancel.");
|
||||
return;
|
||||
}
|
||||
final List<WriteStatus> writeStatus;
|
||||
if (buffer.size() > 0) {
|
||||
writeStatus = new ArrayList<>();
|
||||
|
||||
@@ -22,6 +22,7 @@ import org.apache.hudi.client.FlinkTaskContextSupplier;
|
||||
import org.apache.hudi.client.HoodieFlinkWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieFlinkEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.operator.event.BatchWriteSuccessEvent;
|
||||
@@ -48,6 +49,7 @@ import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Objects;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.CompletionException;
|
||||
@@ -101,6 +103,11 @@ public class StreamWriteOperatorCoordinator
|
||||
*/
|
||||
private final int parallelism;
|
||||
|
||||
/**
|
||||
* Whether needs to schedule compaction task on finished checkpoints.
|
||||
*/
|
||||
private final boolean needsScheduleCompaction;
|
||||
|
||||
/**
|
||||
* Constructs a StreamingSinkOperatorCoordinator.
|
||||
*
|
||||
@@ -112,6 +119,7 @@ public class StreamWriteOperatorCoordinator
|
||||
int parallelism) {
|
||||
this.conf = conf;
|
||||
this.parallelism = parallelism;
|
||||
this.needsScheduleCompaction = isNeedsScheduleCompaction();
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -152,6 +160,10 @@ public class StreamWriteOperatorCoordinator
|
||||
public void checkpointComplete(long checkpointId) {
|
||||
// start to commit the instant.
|
||||
checkAndCommitWithRetry();
|
||||
// if async compaction is on, schedule the compaction
|
||||
if (needsScheduleCompaction) {
|
||||
writeClient.scheduleCompaction(Option.empty());
|
||||
}
|
||||
// start new instant.
|
||||
startInstant();
|
||||
}
|
||||
@@ -202,6 +214,13 @@ public class StreamWriteOperatorCoordinator
|
||||
// Utilities
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
private boolean isNeedsScheduleCompaction() {
|
||||
return this.conf.getString(FlinkOptions.TABLE_TYPE)
|
||||
.toUpperCase(Locale.ROOT)
|
||||
.equals(HoodieTableType.MERGE_ON_READ.name())
|
||||
&& this.conf.getBoolean(FlinkOptions.COMPACTION_ASYNC_ENABLED);
|
||||
}
|
||||
|
||||
@SuppressWarnings("rawtypes")
|
||||
private void initWriteClient() {
|
||||
writeClient = new HoodieFlinkWriteClient(
|
||||
|
||||
@@ -0,0 +1,94 @@
|
||||
/*
|
||||
* 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.operator.compact;
|
||||
|
||||
import org.apache.hudi.client.FlinkTaskContextSupplier;
|
||||
import org.apache.hudi.client.HoodieFlinkWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieFlinkEngineContext;
|
||||
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||
import org.apache.hudi.common.model.CompactionOperation;
|
||||
import org.apache.hudi.table.HoodieFlinkCopyOnWriteTable;
|
||||
import org.apache.hudi.table.action.compact.HoodieFlinkMergeOnReadTableCompactor;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
|
||||
import org.apache.flink.util.Collector;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Function to execute the actual compaction task assigned by the compaction plan task.
|
||||
* In order to execute scalable, the input should shuffle by the compact event {@link CompactionPlanEvent}.
|
||||
*/
|
||||
public class CompactFunction extends KeyedProcessFunction<Long, CompactionPlanEvent, CompactionCommitEvent> {
|
||||
|
||||
/**
|
||||
* Config options.
|
||||
*/
|
||||
private final Configuration conf;
|
||||
|
||||
/**
|
||||
* Write Client.
|
||||
*/
|
||||
private transient HoodieFlinkWriteClient writeClient;
|
||||
|
||||
/**
|
||||
* Id of current subtask.
|
||||
*/
|
||||
private int taskID;
|
||||
|
||||
public CompactFunction(Configuration conf) {
|
||||
this.conf = conf;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open(Configuration parameters) throws Exception {
|
||||
this.taskID = getRuntimeContext().getIndexOfThisSubtask();
|
||||
initWriteClient();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void processElement(CompactionPlanEvent event, Context context, Collector<CompactionCommitEvent> collector) throws Exception {
|
||||
final String instantTime = event.getCompactionInstantTime();
|
||||
final CompactionOperation compactionOperation = event.getOperation();
|
||||
|
||||
HoodieFlinkMergeOnReadTableCompactor compactor = new HoodieFlinkMergeOnReadTableCompactor();
|
||||
List<WriteStatus> writeStatuses = compactor.compact(
|
||||
new HoodieFlinkCopyOnWriteTable<>(
|
||||
this.writeClient.getConfig(),
|
||||
this.writeClient.getEngineContext(),
|
||||
this.writeClient.getHoodieTable().getMetaClient()),
|
||||
this.writeClient.getHoodieTable().getMetaClient(),
|
||||
this.writeClient.getConfig(),
|
||||
compactionOperation,
|
||||
instantTime);
|
||||
collector.collect(new CompactionCommitEvent(instantTime, writeStatuses, taskID));
|
||||
}
|
||||
|
||||
private void initWriteClient() {
|
||||
HoodieFlinkEngineContext context =
|
||||
new HoodieFlinkEngineContext(
|
||||
new SerializableConfiguration(StreamerUtil.getHadoopConf()),
|
||||
new FlinkTaskContextSupplier(getRuntimeContext()));
|
||||
|
||||
writeClient = new HoodieFlinkWriteClient<>(context, StreamerUtil.getHoodieClientConfig(conf));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,62 @@
|
||||
/*
|
||||
* 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.operator.compact;
|
||||
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Represents a commit event from the compaction task {@link CompactFunction}.
|
||||
*/
|
||||
public class CompactionCommitEvent implements Serializable {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
/**
|
||||
* The compaction commit instant time.
|
||||
*/
|
||||
private final String instant;
|
||||
/**
|
||||
* The write statuses.
|
||||
*/
|
||||
private final List<WriteStatus> writeStatuses;
|
||||
/**
|
||||
* The compaction task identifier.
|
||||
*/
|
||||
private final int taskID;
|
||||
|
||||
public CompactionCommitEvent(String instant, List<WriteStatus> writeStatuses, int taskID) {
|
||||
this.instant = instant;
|
||||
this.writeStatuses = writeStatuses;
|
||||
this.taskID = taskID;
|
||||
}
|
||||
|
||||
public String getInstant() {
|
||||
return instant;
|
||||
}
|
||||
|
||||
public List<WriteStatus> getWriteStatuses() {
|
||||
return writeStatuses;
|
||||
}
|
||||
|
||||
public int getTaskID() {
|
||||
return taskID;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,150 @@
|
||||
/*
|
||||
* 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.operator.compact;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.client.FlinkTaskContextSupplier;
|
||||
import org.apache.hudi.client.HoodieFlinkWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieFlinkEngineContext;
|
||||
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Function to check and commit the compaction action.
|
||||
*
|
||||
* <p> Each time after receiving a compaction commit event {@link CompactionCommitEvent},
|
||||
* it loads and checks the compaction plan {@link HoodieCompactionPlan},
|
||||
* if all the compaction operations {@link org.apache.hudi.common.model.CompactionOperation}
|
||||
* of the plan are finished, tries to commit the compaction action.
|
||||
*/
|
||||
public class CompactionCommitSink extends RichSinkFunction<CompactionCommitEvent> {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(CompactionCommitSink.class);
|
||||
|
||||
/**
|
||||
* Config options.
|
||||
*/
|
||||
private final Configuration conf;
|
||||
|
||||
/**
|
||||
* Write Client.
|
||||
*/
|
||||
private transient HoodieFlinkWriteClient writeClient;
|
||||
|
||||
/**
|
||||
* Buffer to collect the event from each compact task {@code CompactFunction}.
|
||||
*/
|
||||
private transient List<CompactionCommitEvent> commitBuffer;
|
||||
|
||||
/**
|
||||
* Current on-going compaction instant time.
|
||||
*/
|
||||
private String compactionInstantTime;
|
||||
|
||||
public CompactionCommitSink(Configuration conf) {
|
||||
this.conf = conf;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open(Configuration parameters) throws Exception {
|
||||
super.open(parameters);
|
||||
initWriteClient();
|
||||
this.commitBuffer = new ArrayList<>();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void invoke(CompactionCommitEvent event, Context context) throws Exception {
|
||||
if (compactionInstantTime == null) {
|
||||
compactionInstantTime = event.getInstant();
|
||||
} else if (!event.getInstant().equals(compactionInstantTime)) {
|
||||
// last compaction still not finish, rolls it back
|
||||
HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(this.compactionInstantTime);
|
||||
writeClient.rollbackInflightCompaction(inflightInstant);
|
||||
this.compactionInstantTime = event.getInstant();
|
||||
}
|
||||
this.commitBuffer.add(event);
|
||||
commitIfNecessary();
|
||||
}
|
||||
|
||||
/**
|
||||
* Condition to commit: the commit buffer has equal size with the compaction plan operations
|
||||
* and all the compact commit event {@link CompactionCommitEvent} has the same compaction instant time.
|
||||
*/
|
||||
private void commitIfNecessary() throws IOException {
|
||||
HoodieCompactionPlan compactionPlan = CompactionUtils.getCompactionPlan(
|
||||
this.writeClient.getHoodieTable().getMetaClient(), compactionInstantTime);
|
||||
boolean isReady = compactionPlan.getOperations().size() == commitBuffer.size()
|
||||
&& commitBuffer.stream().allMatch(event -> event != null && Objects.equals(event.getInstant(), compactionInstantTime));
|
||||
if (!isReady) {
|
||||
return;
|
||||
}
|
||||
List<WriteStatus> statuses = this.commitBuffer.stream()
|
||||
.map(CompactionCommitEvent::getWriteStatuses)
|
||||
.flatMap(Collection::stream)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
if (this.writeClient.getConfig().shouldAutoCommit()) {
|
||||
// Prepare the commit metadata.
|
||||
List<HoodieWriteStat> updateStatusMap = statuses.stream().map(WriteStatus::getStat).collect(Collectors.toList());
|
||||
HoodieCommitMetadata metadata = new HoodieCommitMetadata(true);
|
||||
for (HoodieWriteStat stat : updateStatusMap) {
|
||||
metadata.addWriteStat(stat.getPartitionPath(), stat);
|
||||
}
|
||||
metadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, writeClient.getConfig().getSchema());
|
||||
this.writeClient.completeCompaction(
|
||||
metadata, statuses, this.writeClient.getHoodieTable(), compactionInstantTime);
|
||||
}
|
||||
// commit the compaction
|
||||
this.writeClient.commitCompaction(compactionInstantTime, statuses, Option.empty());
|
||||
// reset the status
|
||||
reset();
|
||||
}
|
||||
|
||||
private void reset() {
|
||||
this.commitBuffer.clear();
|
||||
this.compactionInstantTime = null;
|
||||
}
|
||||
|
||||
private void initWriteClient() {
|
||||
HoodieFlinkEngineContext context =
|
||||
new HoodieFlinkEngineContext(
|
||||
new SerializableConfiguration(StreamerUtil.getHadoopConf()),
|
||||
new FlinkTaskContextSupplier(getRuntimeContext()));
|
||||
|
||||
writeClient = new HoodieFlinkWriteClient<>(context, StreamerUtil.getHoodieClientConfig(this.conf));
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,47 @@
|
||||
/*
|
||||
* 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.operator.compact;
|
||||
|
||||
import org.apache.hudi.common.model.CompactionOperation;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
/**
|
||||
* Represents a compact command from the compaction plan task {@link CompactionPlanOperator}.
|
||||
*/
|
||||
public class CompactionPlanEvent implements Serializable {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
private final String compactionInstantTime;
|
||||
|
||||
private final CompactionOperation operation;
|
||||
|
||||
public CompactionPlanEvent(String instantTime, CompactionOperation operation) {
|
||||
this.compactionInstantTime = instantTime;
|
||||
this.operation = operation;
|
||||
}
|
||||
|
||||
public String getCompactionInstantTime() {
|
||||
return compactionInstantTime;
|
||||
}
|
||||
|
||||
public CompactionOperation getOperation() {
|
||||
return operation;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,146 @@
|
||||
/*
|
||||
* 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.operator.compact;
|
||||
|
||||
import org.apache.hudi.avro.model.HoodieCompactionPlan;
|
||||
import org.apache.hudi.client.FlinkTaskContextSupplier;
|
||||
import org.apache.hudi.client.HoodieFlinkWriteClient;
|
||||
import org.apache.hudi.client.common.HoodieFlinkEngineContext;
|
||||
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||
import org.apache.hudi.common.model.CompactionOperation;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.CompactionUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.table.HoodieFlinkTable;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.flink.annotation.VisibleForTesting;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
|
||||
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
|
||||
import org.apache.flink.streaming.api.operators.Output;
|
||||
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
||||
import static java.util.stream.Collectors.toList;
|
||||
|
||||
/**
|
||||
* Operator that generates the compaction plan with pluggable strategies on finished checkpoints.
|
||||
*
|
||||
* <p>It should be singleton to avoid conflicts.
|
||||
*/
|
||||
public class CompactionPlanOperator extends AbstractStreamOperator<CompactionPlanEvent>
|
||||
implements OneInputStreamOperator<Object, CompactionPlanEvent> {
|
||||
|
||||
/**
|
||||
* Config options.
|
||||
*/
|
||||
private final Configuration conf;
|
||||
|
||||
/**
|
||||
* Write Client.
|
||||
*/
|
||||
private transient HoodieFlinkWriteClient writeClient;
|
||||
|
||||
/**
|
||||
* Compaction instant time.
|
||||
*/
|
||||
private String compactionInstantTime;
|
||||
|
||||
public CompactionPlanOperator(Configuration conf) {
|
||||
this.conf = conf;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void open() throws Exception {
|
||||
super.open();
|
||||
initWriteClient();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void processElement(StreamRecord<Object> streamRecord) {
|
||||
// no operation
|
||||
}
|
||||
|
||||
@Override
|
||||
public void notifyCheckpointComplete(long checkpointId) throws IOException {
|
||||
HoodieFlinkTable<?> table = writeClient.getHoodieTable();
|
||||
// the last instant takes the highest priority.
|
||||
Option<HoodieInstant> compactionInstant = table.getActiveTimeline().filterPendingCompactionTimeline().lastInstant();
|
||||
String compactionInstantTime = compactionInstant.isPresent() ? compactionInstant.get().getTimestamp() : null;
|
||||
if (compactionInstantTime == null) {
|
||||
// do nothing.
|
||||
LOG.info("No compaction plan for checkpoint " + checkpointId);
|
||||
return;
|
||||
}
|
||||
if (this.compactionInstantTime != null
|
||||
&& Objects.equals(this.compactionInstantTime, compactionInstantTime)) {
|
||||
// do nothing
|
||||
LOG.info("Duplicate scheduling for compaction instant: " + compactionInstantTime + ", ignore");
|
||||
return;
|
||||
}
|
||||
// generate compaction plan
|
||||
// should support configurable commit metadata
|
||||
HoodieCompactionPlan compactionPlan = CompactionUtils.getCompactionPlan(
|
||||
table.getMetaClient(), compactionInstantTime);
|
||||
|
||||
if (compactionPlan == null || (compactionPlan.getOperations() == null)
|
||||
|| (compactionPlan.getOperations().isEmpty())) {
|
||||
// do nothing.
|
||||
LOG.info("No compaction plan for checkpoint " + checkpointId + " and instant " + compactionInstantTime);
|
||||
} else {
|
||||
this.compactionInstantTime = compactionInstantTime;
|
||||
HoodieInstant instant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime);
|
||||
HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline();
|
||||
if (!pendingCompactionTimeline.containsInstant(instant)) {
|
||||
throw new IllegalStateException(
|
||||
"No Compaction request available at " + compactionInstantTime + " to run compaction");
|
||||
}
|
||||
|
||||
// Mark instant as compaction inflight
|
||||
table.getActiveTimeline().transitionCompactionRequestedToInflight(instant);
|
||||
table.getMetaClient().reloadActiveTimeline();
|
||||
|
||||
List<CompactionOperation> operations = compactionPlan.getOperations().stream()
|
||||
.map(CompactionOperation::convertFromAvroRecordInstance).collect(toList());
|
||||
LOG.info("CompactionPlanFunction compacting " + operations + " files");
|
||||
for (CompactionOperation operation : operations) {
|
||||
output.collect(new StreamRecord<>(new CompactionPlanEvent(compactionInstantTime, operation)));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public void setOutput(Output<StreamRecord<CompactionPlanEvent>> output) {
|
||||
this.output = output;
|
||||
}
|
||||
|
||||
private void initWriteClient() {
|
||||
HoodieFlinkEngineContext context =
|
||||
new HoodieFlinkEngineContext(
|
||||
new SerializableConfiguration(StreamerUtil.getHadoopConf()),
|
||||
new FlinkTaskContextSupplier(getRuntimeContext()));
|
||||
|
||||
writeClient = new HoodieFlinkWriteClient<>(context, StreamerUtil.getHoodieClientConfig(this.conf));
|
||||
}
|
||||
}
|
||||
@@ -26,6 +26,7 @@ import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.common.util.ParquetUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
@@ -136,7 +137,10 @@ public class BucketAssignFunction<K, I, O extends HoodieRecord<?>>
|
||||
this.context = new HoodieFlinkEngineContext(
|
||||
new SerializableConfiguration(this.hadoopConf),
|
||||
new FlinkTaskContextSupplier(getRuntimeContext()));
|
||||
this.bucketAssigner = new BucketAssigner(context, writeConfig);
|
||||
this.bucketAssigner = BucketAssigners.create(
|
||||
HoodieTableType.valueOf(conf.getString(FlinkOptions.TABLE_TYPE)),
|
||||
context,
|
||||
writeConfig);
|
||||
|
||||
// initialize and check the partitions load state
|
||||
loadInitialPartitions();
|
||||
@@ -145,7 +149,7 @@ public class BucketAssignFunction<K, I, O extends HoodieRecord<?>>
|
||||
|
||||
@Override
|
||||
public void snapshotState(FunctionSnapshotContext context) {
|
||||
// no operation
|
||||
this.bucketAssigner.reset();
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -209,7 +213,6 @@ public class BucketAssignFunction<K, I, O extends HoodieRecord<?>>
|
||||
@Override
|
||||
public void notifyCheckpointComplete(long l) {
|
||||
// Refresh the table state when there are new commits.
|
||||
this.bucketAssigner.reset();
|
||||
this.bucketAssigner.refreshTable();
|
||||
checkPartitionsLoaded();
|
||||
}
|
||||
|
||||
@@ -65,7 +65,7 @@ public class BucketAssigner {
|
||||
*/
|
||||
private final HashMap<String, BucketInfo> bucketInfoMap;
|
||||
|
||||
private HoodieTable<?, ?, ?, ?> table;
|
||||
protected HoodieTable<?, ?, ?, ?> table;
|
||||
|
||||
/**
|
||||
* Fink engine context.
|
||||
@@ -75,7 +75,7 @@ public class BucketAssigner {
|
||||
/**
|
||||
* The write config.
|
||||
*/
|
||||
private final HoodieWriteConfig config;
|
||||
protected final HoodieWriteConfig config;
|
||||
|
||||
/**
|
||||
* The average record size.
|
||||
|
||||
@@ -0,0 +1,54 @@
|
||||
/*
|
||||
* 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.operator.partitioner;
|
||||
|
||||
import org.apache.hudi.client.common.HoodieFlinkEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.operator.partitioner.delta.DeltaBucketAssigner;
|
||||
|
||||
/**
|
||||
* Utilities for {@code BucketAssigner}.
|
||||
*/
|
||||
public abstract class BucketAssigners {
|
||||
|
||||
private BucketAssigners() {}
|
||||
|
||||
/**
|
||||
* Creates a {@code BucketAssigner}.
|
||||
*
|
||||
* @param tableType The table type
|
||||
* @param context The engine context
|
||||
* @param config The configuration
|
||||
* @return the bucket assigner instance
|
||||
*/
|
||||
public static BucketAssigner create(
|
||||
HoodieTableType tableType,
|
||||
HoodieFlinkEngineContext context,
|
||||
HoodieWriteConfig config) {
|
||||
switch (tableType) {
|
||||
case COPY_ON_WRITE:
|
||||
return new BucketAssigner(context, config);
|
||||
case MERGE_ON_READ:
|
||||
return new DeltaBucketAssigner(context, config);
|
||||
default:
|
||||
throw new AssertionError();
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,114 @@
|
||||
/*
|
||||
* 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.operator.partitioner.delta;
|
||||
|
||||
import org.apache.hudi.client.common.HoodieFlinkEngineContext;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.FileSlice;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
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.operator.partitioner.BucketAssigner;
|
||||
import org.apache.hudi.table.action.commit.SmallFile;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* BucketAssigner for MERGE_ON_READ table type, this allows auto correction of small parquet files to larger ones
|
||||
* without the need for an index in the logFile.
|
||||
*
|
||||
* <p>Note: assumes the index can always index log files for Flink write.
|
||||
*/
|
||||
public class DeltaBucketAssigner extends BucketAssigner {
|
||||
public DeltaBucketAssigner(HoodieFlinkEngineContext context, HoodieWriteConfig config) {
|
||||
super(context, config);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected List<SmallFile> getSmallFiles(String partitionPath) {
|
||||
// smallFiles only for partitionPath
|
||||
List<SmallFile> smallFileLocations = new ArrayList<>();
|
||||
|
||||
// Init here since this class (and member variables) might not have been initialized
|
||||
HoodieTimeline commitTimeline = table.getCompletedCommitsTimeline();
|
||||
|
||||
// Find out all eligible small file slices
|
||||
if (!commitTimeline.empty()) {
|
||||
HoodieInstant latestCommitTime = commitTimeline.lastInstant().get();
|
||||
// find smallest file in partition and append to it
|
||||
List<FileSlice> allSmallFileSlices = new ArrayList<>();
|
||||
// If we can index log files, we can add more inserts to log files for fileIds including those under
|
||||
// pending compaction.
|
||||
List<FileSlice> allFileSlices =
|
||||
table.getSliceView().getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp(), true)
|
||||
.collect(Collectors.toList());
|
||||
for (FileSlice fileSlice : allFileSlices) {
|
||||
if (isSmallFile(fileSlice)) {
|
||||
allSmallFileSlices.add(fileSlice);
|
||||
}
|
||||
}
|
||||
// Create SmallFiles from the eligible file slices
|
||||
for (FileSlice smallFileSlice : allSmallFileSlices) {
|
||||
SmallFile sf = new SmallFile();
|
||||
if (smallFileSlice.getBaseFile().isPresent()) {
|
||||
// TODO : Move logic of file name, file id, base commit time handling inside file slice
|
||||
String filename = smallFileSlice.getBaseFile().get().getFileName();
|
||||
sf.location = new HoodieRecordLocation(FSUtils.getCommitTime(filename), FSUtils.getFileId(filename));
|
||||
sf.sizeBytes = getTotalFileSize(smallFileSlice);
|
||||
smallFileLocations.add(sf);
|
||||
} else {
|
||||
HoodieLogFile logFile = smallFileSlice.getLogFiles().findFirst().get();
|
||||
sf.location = new HoodieRecordLocation(FSUtils.getBaseCommitTimeFromLogPath(logFile.getPath()),
|
||||
FSUtils.getFileIdFromLogPath(logFile.getPath()));
|
||||
sf.sizeBytes = getTotalFileSize(smallFileSlice);
|
||||
smallFileLocations.add(sf);
|
||||
}
|
||||
}
|
||||
}
|
||||
return smallFileLocations;
|
||||
}
|
||||
|
||||
private long getTotalFileSize(FileSlice fileSlice) {
|
||||
if (!fileSlice.getBaseFile().isPresent()) {
|
||||
return convertLogFilesSizeToExpectedParquetSize(fileSlice.getLogFiles().collect(Collectors.toList()));
|
||||
} else {
|
||||
return fileSlice.getBaseFile().get().getFileSize()
|
||||
+ convertLogFilesSizeToExpectedParquetSize(fileSlice.getLogFiles().collect(Collectors.toList()));
|
||||
}
|
||||
}
|
||||
|
||||
private boolean isSmallFile(FileSlice fileSlice) {
|
||||
long totalSize = getTotalFileSize(fileSlice);
|
||||
return totalSize < config.getParquetMaxFileSize();
|
||||
}
|
||||
|
||||
// TODO (NA) : Make this static part of utility
|
||||
public long convertLogFilesSizeToExpectedParquetSize(List<HoodieLogFile> hoodieLogFiles) {
|
||||
long totalSizeOfLogFiles = hoodieLogFiles.stream().map(HoodieLogFile::getFileSize)
|
||||
.filter(size -> size > 0).reduce(Long::sum).orElse(0L);
|
||||
// Here we assume that if there is no base parquet file, all log files contain only inserts.
|
||||
// We can then just get the parquet equivalent size of these log files, compare that with
|
||||
// {@link config.getParquetMaxFileSize()} and decide if there is scope to insert more rows
|
||||
return (long) (totalSizeOfLogFiles * config.getLogFileToParquetCompressionRatio());
|
||||
}
|
||||
}
|
||||
@@ -36,6 +36,7 @@ import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.keygen.KeyGenerator;
|
||||
import org.apache.hudi.operator.FlinkOptions;
|
||||
import org.apache.hudi.schema.FilebasedSchemaProvider;
|
||||
import org.apache.hudi.table.action.compact.CompactionTriggerStrategy;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
@@ -54,6 +55,7 @@ import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.StringReader;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Objects;
|
||||
import java.util.Properties;
|
||||
|
||||
@@ -229,6 +231,10 @@ public class StreamerUtil {
|
||||
.withCompactionConfig(
|
||||
HoodieCompactionConfig.newBuilder()
|
||||
.withPayloadClass(conf.getString(FlinkOptions.PAYLOAD_CLASS))
|
||||
.withInlineCompactionTriggerStrategy(
|
||||
CompactionTriggerStrategy.valueOf(conf.getString(FlinkOptions.COMPACTION_TRIGGER_STRATEGY).toUpperCase(Locale.ROOT)))
|
||||
.withMaxNumDeltaCommitsBeforeCompaction(conf.getInteger(FlinkOptions.COMPACTION_DELTA_COMMITS))
|
||||
.withMaxDeltaSecondsBeforeCompaction(conf.getInteger(FlinkOptions.COMPACTION_DELTA_SECONDS))
|
||||
.build())
|
||||
.forTable(conf.getString(FlinkOptions.TABLE_NAME))
|
||||
.withAutoCommit(false)
|
||||
|
||||
Reference in New Issue
Block a user