Skip to content

Commit

Permalink
[HUDI-5228] Flink table service job fs view conf overwrites the one o…
Browse files Browse the repository at this point in the history
…f writing job (#7214)
  • Loading branch information
danny0405 authored Nov 16, 2022
1 parent d3f9577 commit dc5cc08
Show file tree
Hide file tree
Showing 28 changed files with 304 additions and 231 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.sink.utils.NonThrownExecutor;
import org.apache.hudi.util.StreamerUtil;
import org.apache.hudi.util.FlinkWriteClients;

import org.apache.flink.api.common.functions.AbstractRichFunction;
import org.apache.flink.api.common.state.CheckpointListener;
Expand Down Expand Up @@ -60,7 +60,7 @@ public CleanFunction(Configuration conf) {
@Override
public void open(Configuration parameters) throws Exception {
super.open(parameters);
this.writeClient = StreamerUtil.createWriteClient(conf, getRuntimeContext());
this.writeClient = FlinkWriteClients.createWriteClient(conf, getRuntimeContext());
this.executor = NonThrownExecutor.builder(LOG).waitForTasksFinish(true).build();
String instantTime = HoodieActiveTimeline.createNewInstantTime();
LOG.info(String.format("exec clean with instant time %s...", instantTime));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
import org.apache.hudi.sink.utils.NonThrownExecutor;
import org.apache.hudi.util.ClusteringUtil;
import org.apache.hudi.util.CompactionUtil;
import org.apache.hudi.util.FlinkWriteClients;
import org.apache.hudi.util.StreamerUtil;

import org.apache.flink.annotation.VisibleForTesting;
Expand Down Expand Up @@ -179,7 +180,7 @@ public void start() throws Exception {
this.metaClient = initTableIfNotExists(this.conf);
this.ckpMetadata = initCkpMetadata(this.metaClient);
// the write client must create after the table creation
this.writeClient = StreamerUtil.createWriteClient(conf);
this.writeClient = FlinkWriteClients.createWriteClient(conf);
initMetadataTable(this.writeClient);
this.tableState = TableState.create(conf);
// start the executor
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.format.FormatUtils;
import org.apache.hudi.util.FlinkTables;
import org.apache.hudi.util.StreamerUtil;
import org.apache.hudi.util.FlinkWriteClients;

import org.apache.avro.Schema;
import org.apache.flink.annotation.VisibleForTesting;
Expand Down Expand Up @@ -125,7 +125,7 @@ public void initializeState(StateInitializationContext context) throws Exception
}

this.hadoopConf = HadoopConfigurations.getHadoopConf(this.conf);
this.writeConfig = StreamerUtil.getHoodieClientConfig(this.conf, true);
this.writeConfig = FlinkWriteClients.getHoodieClientConfig(this.conf, true);
this.hoodieTable = FlinkTables.createTable(writeConfig, hadoopConf, getRuntimeContext());
this.ckpMetadata = CkpMetadata.getInstance(hoodieTable.getMetaClient().getFs(), this.writeConfig.getBasePath());
this.aggregateManager = getRuntimeContext().getGlobalAggregateManager();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@
import org.apache.hudi.sink.event.WriteMetadataEvent;
import org.apache.hudi.sink.meta.CkpMetadata;
import org.apache.hudi.sink.utils.TimeWait;
import org.apache.hudi.util.StreamerUtil;
import org.apache.hudi.util.FlinkWriteClients;

import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.operators.coordination.OperatorEvent;
Expand Down Expand Up @@ -112,7 +112,7 @@ public BulkInsertWriteFunction(Configuration config, RowType rowType) {
@Override
public void open(Configuration parameters) throws IOException {
this.taskID = getRuntimeContext().getIndexOfThisSubtask();
this.writeClient = StreamerUtil.createWriteClient(this.config, getRuntimeContext());
this.writeClient = FlinkWriteClients.createWriteClient(this.config, getRuntimeContext());
this.ckpMetadata = CkpMetadata.getInstance(config);
this.initInstant = lastPendingInstant();
sendBootstrapEvent();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
import org.apache.hudi.table.HoodieFlinkTable;
import org.apache.hudi.table.action.HoodieWriteMetadata;
import org.apache.hudi.util.CompactionUtil;
import org.apache.hudi.util.FlinkWriteClients;
import org.apache.hudi.util.StreamerUtil;

import org.apache.flink.configuration.Configuration;
Expand Down Expand Up @@ -87,7 +88,7 @@ public ClusteringCommitSink(Configuration conf) {
public void open(Configuration parameters) throws Exception {
super.open(parameters);
if (writeClient == null) {
this.writeClient = StreamerUtil.createWriteClient(conf, getRuntimeContext());
this.writeClient = FlinkWriteClients.createWriteClient(conf, getRuntimeContext());
}
this.commitBuffer = new HashMap<>();
this.table = writeClient.getHoodieTable();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@
import org.apache.hudi.table.HoodieFlinkTable;
import org.apache.hudi.util.AvroSchemaConverter;
import org.apache.hudi.util.AvroToRowDataConverters;
import org.apache.hudi.util.StreamerUtil;
import org.apache.hudi.util.FlinkWriteClients;

import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
Expand Down Expand Up @@ -144,8 +144,8 @@ public void open() throws Exception {
super.open();

this.taskID = getRuntimeContext().getIndexOfThisSubtask();
this.writeConfig = StreamerUtil.getHoodieClientConfig(this.conf);
this.writeClient = StreamerUtil.createWriteClient(conf, getRuntimeContext());
this.writeConfig = FlinkWriteClients.getHoodieClientConfig(this.conf);
this.writeClient = FlinkWriteClients.createWriteClient(conf, getRuntimeContext());
this.table = writeClient.getHoodieTable();

this.schema = AvroSchemaConverter.convertToSchema(rowType);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@
import org.apache.hudi.table.HoodieFlinkTable;
import org.apache.hudi.util.ClusteringUtil;
import org.apache.hudi.util.FlinkTables;
import org.apache.hudi.util.StreamerUtil;
import org.apache.hudi.util.FlinkWriteClients;

import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.configuration.Configuration;
Expand Down Expand Up @@ -68,7 +68,7 @@ public void open() throws Exception {
// when starting up, rolls back all the inflight clustering instants if there exists,
// these instants are in priority for scheduling task because the clustering instants are
// scheduled from earliest(FIFO sequence).
ClusteringUtil.rollbackClustering(table, StreamerUtil.createWriteClient(conf, getRuntimeContext()));
ClusteringUtil.rollbackClustering(table, FlinkWriteClients.createWriteClient(conf, getRuntimeContext()));
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
import org.apache.hudi.util.AvroSchemaConverter;
import org.apache.hudi.util.ClusteringUtil;
import org.apache.hudi.util.CompactionUtil;
import org.apache.hudi.util.FlinkWriteClients;
import org.apache.hudi.util.StreamerUtil;

import com.beust.jcommander.JCommander;
Expand Down Expand Up @@ -198,7 +199,7 @@ public AsyncClusteringService(FlinkClusteringConfig cfg, Configuration conf, Str
// set table schema
CompactionUtil.setAvroSchema(conf, metaClient);

this.writeClient = StreamerUtil.createWriteClient(conf);
this.writeClient = FlinkWriteClients.createWriteClientV2(conf);
this.writeConfig = writeClient.getConfig();
this.table = writeClient.getHoodieTable();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import org.apache.hudi.sink.event.WriteMetadataEvent;
import org.apache.hudi.sink.meta.CkpMetadata;
import org.apache.hudi.sink.utils.TimeWait;
import org.apache.hudi.util.FlinkWriteClients;
import org.apache.hudi.util.StreamerUtil;

import org.apache.flink.annotation.VisibleForTesting;
Expand Down Expand Up @@ -139,7 +140,7 @@ public AbstractStreamWriteFunction(Configuration config) {
public void initializeState(FunctionInitializationContext context) throws Exception {
this.taskID = getRuntimeContext().getIndexOfThisSubtask();
this.metaClient = StreamerUtil.createMetaClient(this.config);
this.writeClient = StreamerUtil.createWriteClient(this.config, getRuntimeContext());
this.writeClient = FlinkWriteClients.createWriteClient(this.config, getRuntimeContext());
this.writeStatuses = new ArrayList<>();
this.writeMetadataState = context.getOperatorStateStore().getListState(
new ListStateDescriptor<>(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@
import org.apache.hudi.table.HoodieFlinkCopyOnWriteTable;
import org.apache.hudi.table.action.compact.HoodieFlinkMergeOnReadTableCompactor;
import org.apache.hudi.util.CompactionUtil;
import org.apache.hudi.util.StreamerUtil;
import org.apache.hudi.util.FlinkWriteClients;

import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.configuration.Configuration;
Expand Down Expand Up @@ -80,7 +80,7 @@ public CompactFunction(Configuration conf) {
@Override
public void open(Configuration parameters) throws Exception {
this.taskID = getRuntimeContext().getIndexOfThisSubtask();
this.writeClient = StreamerUtil.createWriteClient(conf, getRuntimeContext());
this.writeClient = FlinkWriteClients.createWriteClient(conf, getRuntimeContext());
if (this.asyncCompaction) {
this.executor = NonThrownExecutor.builder(LOG).build();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@
import org.apache.hudi.table.HoodieFlinkTable;
import org.apache.hudi.table.action.compact.CompactHelpers;
import org.apache.hudi.util.CompactionUtil;
import org.apache.hudi.util.StreamerUtil;
import org.apache.hudi.util.FlinkWriteClients;

import org.apache.flink.configuration.Configuration;
import org.slf4j.Logger;
Expand Down Expand Up @@ -91,7 +91,7 @@ public CompactionCommitSink(Configuration conf) {
public void open(Configuration parameters) throws Exception {
super.open(parameters);
if (writeClient == null) {
this.writeClient = StreamerUtil.createWriteClient(conf, getRuntimeContext());
this.writeClient = FlinkWriteClients.createWriteClient(conf, getRuntimeContext());
}
this.commitBuffer = new HashMap<>();
this.compactionPlanCache = new HashMap<>();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import org.apache.hudi.sink.compact.strategy.CompactionPlanStrategies;
import org.apache.hudi.table.HoodieFlinkTable;
import org.apache.hudi.util.CompactionUtil;
import org.apache.hudi.util.FlinkWriteClients;
import org.apache.hudi.util.StreamerUtil;

import com.beust.jcommander.JCommander;
Expand Down Expand Up @@ -187,7 +188,7 @@ public AsyncCompactionService(FlinkCompactionConfig cfg, Configuration conf, Str
// infer changelog mode
CompactionUtil.inferChangelogMode(conf, metaClient);

this.writeClient = StreamerUtil.createWriteClient(conf);
this.writeClient = FlinkWriteClients.createWriteClientV2(conf);
this.writeConfig = writeClient.getConfig();
this.table = writeClient.getHoodieTable();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@
import org.apache.hudi.sink.bootstrap.IndexRecord;
import org.apache.hudi.sink.utils.PayloadCreation;
import org.apache.hudi.table.action.commit.BucketInfo;
import org.apache.hudi.util.StreamerUtil;
import org.apache.hudi.util.FlinkWriteClients;

import org.apache.flink.api.common.state.CheckpointListener;
import org.apache.flink.api.common.state.StateTtlConfig;
Expand Down Expand Up @@ -115,7 +115,7 @@ public BucketAssignFunction(Configuration conf) {
@Override
public void open(Configuration parameters) throws Exception {
super.open(parameters);
HoodieWriteConfig writeConfig = StreamerUtil.getHoodieClientConfig(this.conf, true);
HoodieWriteConfig writeConfig = FlinkWriteClients.getHoodieClientConfig(this.conf, true);
HoodieFlinkEngineContext context = new HoodieFlinkEngineContext(
new SerializableConfiguration(HadoopConfigurations.getHadoopConf(this.conf)),
new FlinkTaskContextSupplier(getRuntimeContext()));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@
import org.apache.hudi.configuration.HadoopConfigurations;
import org.apache.hudi.exception.HoodieMetadataException;
import org.apache.hudi.util.AvroSchemaConverter;
import org.apache.hudi.util.FlinkWriteClients;
import org.apache.hudi.util.StreamerUtil;

import org.apache.avro.Schema;
Expand Down Expand Up @@ -555,8 +556,8 @@ private Map<String, String> applyOptionsHook(String tablePath, Map<String, Strin
private HoodieFlinkWriteClient<?> createWriteClient(
Map<String, String> options,
String tablePathStr,
ObjectPath tablePath) throws IOException {
return StreamerUtil.createWriteClient(
ObjectPath tablePath) {
return FlinkWriteClients.createWriteClientV2(
Configuration.fromMap(options)
.set(FlinkOptions.TABLE_NAME, tablePath.getObjectName())
.set(FlinkOptions.SOURCE_AVRO_SCHEMA,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
import org.apache.hudi.table.format.FilePathUtils;
import org.apache.hudi.util.AvroSchemaConverter;
import org.apache.hudi.util.DataTypeUtils;
import org.apache.hudi.util.FlinkWriteClients;
import org.apache.hudi.util.StreamerUtil;

import org.apache.avro.Schema;
Expand Down Expand Up @@ -963,11 +964,11 @@ private Map<String, String> supplementOptions(

private HoodieFlinkWriteClient<?> createWriteClient(
ObjectPath tablePath,
CatalogBaseTable table) throws Exception {
CatalogBaseTable table) {
Map<String, String> options = table.getOptions();
// enable auto-commit though ~
options.put(HoodieWriteConfig.AUTO_COMMIT_ENABLE.key(), "true");
return StreamerUtil.createWriteClient(
return FlinkWriteClients.createWriteClientV2(
Configuration.fromMap(options)
.set(FlinkOptions.TABLE_NAME, tablePath.getObjectName())
.set(FlinkOptions.SOURCE_AVRO_SCHEMA,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.hadoop.config.HoodieRealtimeConfig;
import org.apache.hudi.table.format.mor.MergeOnReadInputSplit;
import org.apache.hudi.util.FlinkWriteClients;
import org.apache.hudi.util.StreamerUtil;

import org.apache.avro.Schema;
Expand Down Expand Up @@ -143,7 +144,7 @@ public static HoodieMergedLogRecordScanner logScanner(
Schema logSchema,
org.apache.flink.configuration.Configuration flinkConf,
Configuration hadoopConf) {
HoodieWriteConfig writeConfig = StreamerUtil.getHoodieClientConfig(flinkConf);
HoodieWriteConfig writeConfig = FlinkWriteClients.getHoodieClientConfig(flinkConf);
FileSystem fs = FSUtils.getFs(split.getTablePath(), hadoopConf);
return HoodieMergedLogRecordScanner.newBuilder()
.withFileSystem(fs)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@
import org.apache.hudi.table.format.mor.MergeOnReadInputSplit;
import org.apache.hudi.table.format.mor.MergeOnReadTableState;
import org.apache.hudi.util.AvroToRowDataConverters;
import org.apache.hudi.util.FlinkWriteClients;
import org.apache.hudi.util.RowDataProjection;
import org.apache.hudi.util.StreamerUtil;

Expand Down Expand Up @@ -609,7 +610,7 @@ public ImageManager(
this.serializer = new RowDataSerializer(rowType);
this.splitIteratorFunc = splitIteratorFunc;
this.cache = new TreeMap<>();
this.writeConfig = StreamerUtil.getHoodieClientConfig(flinkConf);
this.writeConfig = FlinkWriteClients.getHoodieClientConfig(flinkConf);
}

public ExternalSpillableMap<String, byte[]> getOrLoadImages(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,6 @@
import org.apache.flink.configuration.Configuration;

import static org.apache.hudi.configuration.HadoopConfigurations.getHadoopConf;
import static org.apache.hudi.util.StreamerUtil.getHoodieClientConfig;

/**
* Utilities for {@link org.apache.hudi.table.HoodieFlinkTable}.
Expand All @@ -46,7 +45,7 @@ public static HoodieFlinkTable<?> createTable(Configuration conf, RuntimeContext
HoodieFlinkEngineContext context = new HoodieFlinkEngineContext(
new SerializableConfiguration(getHadoopConf(conf)),
new FlinkTaskContextSupplier(runtimeContext));
HoodieWriteConfig writeConfig = getHoodieClientConfig(conf, true);
HoodieWriteConfig writeConfig = FlinkWriteClients.getHoodieClientConfig(conf, true);
return HoodieFlinkTable.create(writeConfig, context);
}

Expand All @@ -71,7 +70,7 @@ public static HoodieFlinkTable<?> createTable(
* <p>This expects to be used by driver.
*/
public static HoodieFlinkTable<?> createTable(Configuration conf) {
HoodieWriteConfig writeConfig = StreamerUtil.getHoodieClientConfig(conf, true, false);
HoodieWriteConfig writeConfig = FlinkWriteClients.getHoodieClientConfig(conf, true, false);
return HoodieFlinkTable.create(writeConfig, HoodieFlinkEngineContext.DEFAULT);
}
}
Loading

0 comments on commit dc5cc08

Please sign in to comment.