[HUDI-2258] Metadata table for flink (#3381)
This commit is contained in:
@@ -76,6 +76,22 @@ public class FlinkOptions extends HoodieConfig {
|
||||
.withDescription("The default partition name in case the dynamic partition"
|
||||
+ " column value is null/empty string");
|
||||
|
||||
// ------------------------------------------------------------------------
|
||||
// Metadata table Options
|
||||
// ------------------------------------------------------------------------
|
||||
|
||||
public static final ConfigOption<Boolean> METADATA_ENABLED = ConfigOptions
|
||||
.key("metadata.enabled")
|
||||
.booleanType()
|
||||
.defaultValue(false)
|
||||
.withDescription("Enable the internal metadata table which serves table metadata like level file listings, default false");
|
||||
|
||||
public static final ConfigOption<Integer> METADATA_COMPACTION_DELTA_COMMITS = ConfigOptions
|
||||
.key("metadata.compaction.delta_commits")
|
||||
.intType()
|
||||
.defaultValue(24)
|
||||
.withDescription("Max delta commits for metadata table to trigger compaction, default 24");
|
||||
|
||||
// ------------------------------------------------------------------------
|
||||
// Index Options
|
||||
// ------------------------------------------------------------------------
|
||||
|
||||
@@ -324,13 +324,7 @@ public class StreamWriteFunction<K, I, O>
|
||||
}
|
||||
|
||||
private void sendBootstrapEvent() {
|
||||
WriteMetadataEvent event = WriteMetadataEvent.builder()
|
||||
.taskID(taskID)
|
||||
.writeStatus(Collections.emptyList())
|
||||
.instantTime("")
|
||||
.bootstrap(true)
|
||||
.build();
|
||||
this.eventGateway.sendEventToCoordinator(event);
|
||||
this.eventGateway.sendEventToCoordinator(WriteMetadataEvent.emptyBootstrap(taskID));
|
||||
LOG.info("Send bootstrap write metadata event to coordinator, task[{}].", taskID);
|
||||
}
|
||||
|
||||
|
||||
@@ -94,7 +94,7 @@ public class StreamWriteOperatorCoordinator
|
||||
/**
|
||||
* Current REQUESTED instant, for validation.
|
||||
*/
|
||||
private volatile String instant = "";
|
||||
private volatile String instant = WriteMetadataEvent.BOOTSTRAP_INSTANT;
|
||||
|
||||
/**
|
||||
* Event buffer for one round of checkpointing. When all the elements are non-null and have the same
|
||||
@@ -107,11 +107,6 @@ public class StreamWriteOperatorCoordinator
|
||||
*/
|
||||
private final int parallelism;
|
||||
|
||||
/**
|
||||
* Whether to schedule compaction plan on finished checkpoints.
|
||||
*/
|
||||
private final boolean scheduleCompaction;
|
||||
|
||||
/**
|
||||
* A single-thread executor to handle all the asynchronous jobs of the coordinator.
|
||||
*/
|
||||
@@ -127,6 +122,11 @@ public class StreamWriteOperatorCoordinator
|
||||
*/
|
||||
private HiveSyncContext hiveSyncContext;
|
||||
|
||||
/**
|
||||
* A single-thread executor to handle metadata table sync.
|
||||
*/
|
||||
private NonThrownExecutor metadataSyncExecutor;
|
||||
|
||||
/**
|
||||
* The table state.
|
||||
*/
|
||||
@@ -144,23 +144,25 @@ public class StreamWriteOperatorCoordinator
|
||||
this.conf = conf;
|
||||
this.context = context;
|
||||
this.parallelism = context.currentParallelism();
|
||||
this.scheduleCompaction = StreamerUtil.needsScheduleCompaction(conf);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void start() throws Exception {
|
||||
// initialize event buffer
|
||||
reset();
|
||||
this.writeClient = StreamerUtil.createWriteClient(conf, null);
|
||||
this.writeClient = StreamerUtil.createWriteClient(conf);
|
||||
this.tableState = TableState.create(conf);
|
||||
// init table, create it if not exists.
|
||||
initTableIfNotExists(this.conf);
|
||||
// start the executor
|
||||
this.executor = new CoordinatorExecutor(this.context, LOG);
|
||||
// start the executor if required
|
||||
if (conf.getBoolean(FlinkOptions.HIVE_SYNC_ENABLED)) {
|
||||
if (tableState.syncHive) {
|
||||
initHiveSync();
|
||||
}
|
||||
if (tableState.syncMetadata) {
|
||||
initMetadataSync();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -205,13 +207,15 @@ public class StreamWriteOperatorCoordinator
|
||||
final boolean committed = commitInstant(this.instant);
|
||||
if (committed) {
|
||||
// if async compaction is on, schedule the compaction
|
||||
if (scheduleCompaction) {
|
||||
if (tableState.scheduleCompaction) {
|
||||
writeClient.scheduleCompaction(Option.empty());
|
||||
}
|
||||
// start new instant.
|
||||
startInstant();
|
||||
// sync Hive if is enabled
|
||||
syncHiveIfEnabled();
|
||||
// sync metadata if is enabled
|
||||
syncMetadataIfEnabled();
|
||||
}
|
||||
}, "commits the instant %s", this.instant
|
||||
);
|
||||
@@ -263,7 +267,7 @@ public class StreamWriteOperatorCoordinator
|
||||
}
|
||||
|
||||
private void syncHiveIfEnabled() {
|
||||
if (conf.getBoolean(FlinkOptions.HIVE_SYNC_ENABLED)) {
|
||||
if (tableState.syncHive) {
|
||||
this.hiveSyncExecutor.execute(this::syncHive, "sync hive metadata for instant %s", this.instant);
|
||||
}
|
||||
}
|
||||
@@ -275,6 +279,27 @@ public class StreamWriteOperatorCoordinator
|
||||
hiveSyncContext.hiveSyncTool().syncHoodieTable();
|
||||
}
|
||||
|
||||
private void initMetadataSync() {
|
||||
this.metadataSyncExecutor = new NonThrownExecutor(LOG, true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Sync the write metadata to the metadata table.
|
||||
*/
|
||||
private void syncMetadataIfEnabled() {
|
||||
if (tableState.syncMetadata) {
|
||||
this.metadataSyncExecutor.execute(this::syncMetadata,
|
||||
"sync metadata table for instant %s", this.instant);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Sync the write metadata to the metadata table.
|
||||
*/
|
||||
private void syncMetadata() {
|
||||
this.writeClient.syncTableMetadata();
|
||||
}
|
||||
|
||||
private void reset() {
|
||||
this.eventBuffer = new WriteMetadataEvent[this.parallelism];
|
||||
}
|
||||
@@ -324,6 +349,11 @@ public class StreamWriteOperatorCoordinator
|
||||
LOG.info("Recommit instant {}", instant);
|
||||
commitInstant(instant);
|
||||
}
|
||||
if (tableState.syncMetadata) {
|
||||
// initialize metadata table first if enabled
|
||||
// condition: the data set timeline has committed instants
|
||||
syncMetadata();
|
||||
}
|
||||
// starts a new instant
|
||||
startInstant();
|
||||
}, "initialize instant %s", instant);
|
||||
@@ -344,6 +374,8 @@ public class StreamWriteOperatorCoordinator
|
||||
commitInstant(this.instant);
|
||||
// sync Hive if is enabled in batch mode.
|
||||
syncHiveIfEnabled();
|
||||
// sync metadata if is enabled in batch mode.
|
||||
syncMetadataIfEnabled();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -480,6 +512,14 @@ public class StreamWriteOperatorCoordinator
|
||||
this.executor = executor;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public void setMetadataSyncExecutor(NonThrownExecutor executor) throws Exception {
|
||||
if (this.metadataSyncExecutor != null) {
|
||||
this.metadataSyncExecutor.close();
|
||||
}
|
||||
this.metadataSyncExecutor = executor;
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Inner Class
|
||||
// -------------------------------------------------------------------------
|
||||
@@ -513,15 +553,21 @@ public class StreamWriteOperatorCoordinator
|
||||
private static class TableState implements Serializable {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
private final WriteOperationType operationType;
|
||||
private final String commitAction;
|
||||
private final boolean isOverwrite;
|
||||
final WriteOperationType operationType;
|
||||
final String commitAction;
|
||||
final boolean isOverwrite;
|
||||
final boolean scheduleCompaction;
|
||||
final boolean syncHive;
|
||||
final boolean syncMetadata;
|
||||
|
||||
private TableState(Configuration conf) {
|
||||
this.operationType = WriteOperationType.fromValue(conf.getString(FlinkOptions.OPERATION));
|
||||
this.commitAction = CommitUtils.getCommitActionType(this.operationType,
|
||||
HoodieTableType.valueOf(conf.getString(FlinkOptions.TABLE_TYPE).toUpperCase(Locale.ROOT)));
|
||||
this.isOverwrite = WriteOperationType.isOverwrite(this.operationType);
|
||||
this.scheduleCompaction = StreamerUtil.needsScheduleCompaction(conf);
|
||||
this.syncHive = conf.getBoolean(FlinkOptions.HIVE_SYNC_ENABLED);
|
||||
this.syncMetadata = conf.getBoolean(FlinkOptions.METADATA_ENABLED);
|
||||
}
|
||||
|
||||
public static TableState create(Configuration conf) {
|
||||
|
||||
@@ -22,8 +22,7 @@ import org.apache.hudi.client.HoodieFlinkWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.common.model.CompactionOperation;
|
||||
import org.apache.hudi.sink.utils.NonThrownExecutor;
|
||||
import org.apache.hudi.table.HoodieFlinkCopyOnWriteTable;
|
||||
import org.apache.hudi.table.action.compact.HoodieFlinkMergeOnReadTableCompactor;
|
||||
import org.apache.hudi.table.action.compact.FlinkCompactHelpers;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
|
||||
import org.apache.flink.annotation.VisibleForTesting;
|
||||
@@ -99,16 +98,7 @@ public class CompactFunction extends ProcessFunction<CompactionPlanEvent, Compac
|
||||
}
|
||||
|
||||
private void doCompaction(String instantTime, CompactionOperation compactionOperation, Collector<CompactionCommitEvent> collector) throws IOException {
|
||||
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);
|
||||
List<WriteStatus> writeStatuses = FlinkCompactHelpers.compact(writeClient, instantTime, compactionOperation);
|
||||
collector.collect(new CompactionCommitEvent(instantTime, writeStatuses, taskID));
|
||||
}
|
||||
|
||||
|
||||
@@ -24,6 +24,7 @@ import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.flink.runtime.operators.coordination.OperatorEvent;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
||||
@@ -33,6 +34,8 @@ import java.util.Objects;
|
||||
public class WriteMetadataEvent implements OperatorEvent {
|
||||
private static final long serialVersionUID = 1L;
|
||||
|
||||
public static final String BOOTSTRAP_INSTANT = "";
|
||||
|
||||
private List<WriteStatus> writeStatuses;
|
||||
private int taskID;
|
||||
private String instantTime;
|
||||
@@ -57,11 +60,11 @@ public class WriteMetadataEvent implements OperatorEvent {
|
||||
* @param taskID The task ID
|
||||
* @param instantTime The instant time under which to write the data
|
||||
* @param writeStatuses The write statues list
|
||||
* @param lastBatch Whether the event reports the last batch
|
||||
* @param lastBatch Whether the event reports the last batch
|
||||
* within an checkpoint interval,
|
||||
* if true, the whole data set of the checkpoint
|
||||
* has been flushed successfully
|
||||
* @param bootstrap Whether the event comes from the bootstrap
|
||||
* @param bootstrap Whether the event comes from the bootstrap
|
||||
*/
|
||||
private WriteMetadataEvent(
|
||||
int taskID,
|
||||
@@ -79,7 +82,8 @@ public class WriteMetadataEvent implements OperatorEvent {
|
||||
}
|
||||
|
||||
// default constructor for efficient serialization
|
||||
public WriteMetadataEvent() {}
|
||||
public WriteMetadataEvent() {
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the builder for {@link WriteMetadataEvent}.
|
||||
@@ -159,6 +163,25 @@ public class WriteMetadataEvent implements OperatorEvent {
|
||||
return lastBatch && this.instantTime.equals(currentInstant);
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Utilities
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
/**
|
||||
* Creates empty bootstrap event for task {@code taskId}.
|
||||
*
|
||||
* <p>The event indicates that the new instant can start directly,
|
||||
* there is no old instant write statuses to recover.
|
||||
*/
|
||||
public static WriteMetadataEvent emptyBootstrap(int taskId) {
|
||||
return WriteMetadataEvent.builder()
|
||||
.taskID(taskId)
|
||||
.instantTime(BOOTSTRAP_INSTANT)
|
||||
.writeStatus(Collections.emptyList())
|
||||
.bootstrap(true)
|
||||
.build();
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------------
|
||||
// Builder
|
||||
// -------------------------------------------------------------------------
|
||||
|
||||
@@ -78,8 +78,7 @@ public class HiveSyncContext {
|
||||
hiveSyncConfig.partitionFields = Arrays.asList(FilePathUtils.extractPartitionKeys(conf));
|
||||
hiveSyncConfig.partitionValueExtractorClass = conf.getString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS);
|
||||
hiveSyncConfig.useJdbc = conf.getBoolean(FlinkOptions.HIVE_SYNC_USE_JDBC);
|
||||
// needs to support metadata table for flink
|
||||
hiveSyncConfig.useFileListingFromMetadata = false;
|
||||
hiveSyncConfig.useFileListingFromMetadata = conf.getBoolean(FlinkOptions.METADATA_ENABLED);
|
||||
hiveSyncConfig.verifyMetadataFileListing = false;
|
||||
hiveSyncConfig.ignoreExceptions = conf.getBoolean(FlinkOptions.HIVE_SYNC_IGNORE_EXCEPTIONS);
|
||||
hiveSyncConfig.supportTimestamp = conf.getBoolean(FlinkOptions.HIVE_SYNC_SUPPORT_TIMESTAMP);
|
||||
|
||||
@@ -22,6 +22,7 @@ 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.DFSPropertiesConfiguration;
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig;
|
||||
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.engine.EngineType;
|
||||
@@ -170,6 +171,10 @@ public class StreamerUtil {
|
||||
.logFileDataBlockMaxSize(conf.getInteger(FlinkOptions.WRITE_LOG_BLOCK_SIZE) * 1024 * 1024)
|
||||
.logFileMaxSize(conf.getInteger(FlinkOptions.WRITE_LOG_MAX_SIZE) * 1024 * 1024)
|
||||
.build())
|
||||
.withMetadataConfig(HoodieMetadataConfig.newBuilder()
|
||||
.enable(conf.getBoolean(FlinkOptions.METADATA_ENABLED))
|
||||
.withMaxNumDeltaCommitsBeforeCompaction(conf.getInteger(FlinkOptions.METADATA_COMPACTION_DELTA_COMMITS))
|
||||
.build())
|
||||
.withEmbeddedTimelineServerReuseEnabled(true) // make write client embedded timeline service singleton
|
||||
.withAutoCommit(false)
|
||||
.withProps(flinkConf2TypedProperties(FlinkOptions.flatOptions(conf)));
|
||||
@@ -265,11 +270,18 @@ public class StreamerUtil {
|
||||
&& conf.getBoolean(FlinkOptions.COMPACTION_SCHEDULE_ENABLED);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates the meta client.
|
||||
*/
|
||||
public static HoodieTableMetaClient createMetaClient(String basePath) {
|
||||
return HoodieTableMetaClient.builder().setBasePath(basePath).setConf(FlinkClientUtil.getHadoopConf()).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates the meta client.
|
||||
*/
|
||||
public static HoodieTableMetaClient createMetaClient(Configuration conf) {
|
||||
return HoodieTableMetaClient.builder().setBasePath(conf.getString(FlinkOptions.PATH)).setConf(FlinkClientUtil.getHadoopConf()).build();
|
||||
return createMetaClient(conf.getString(FlinkOptions.PATH));
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -284,6 +296,15 @@ public class StreamerUtil {
|
||||
return new HoodieFlinkWriteClient<>(context, getHoodieClientConfig(conf));
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates the Flink write client.
|
||||
*
|
||||
* <p>The task context supplier is a constant: the write token is always '0-1-0'.
|
||||
*/
|
||||
public static HoodieFlinkWriteClient createWriteClient(Configuration conf) {
|
||||
return new HoodieFlinkWriteClient<>(HoodieFlinkEngineContext.DEFAULT, getHoodieClientConfig(conf));
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the median instant time between the given two instant time.
|
||||
*/
|
||||
|
||||
Reference in New Issue
Block a user