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 @@ -62,7 +62,7 @@ public void open(Configuration parameters) throws Exception {
if (conf.getBoolean(FlinkOptions.CLEAN_ASYNC_ENABLED)) {
// do not use the remote filesystem view because the async cleaning service
// local timeline is very probably to fall behind with the remote one.
this.writeClient = StreamerUtil.createWriteClient(conf, getRuntimeContext(), false);
this.writeClient = StreamerUtil.createWriteClient(conf, getRuntimeContext());
this.executor = NonThrownExecutor.builder(LOG).build();
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -120,7 +120,7 @@ public void initializeState(StateInitializationContext context) throws Exception
}

this.hadoopConf = StreamerUtil.getHadoopConf();
this.writeConfig = StreamerUtil.getHoodieClientConfig(this.conf, true);
this.writeConfig = StreamerUtil.getHoodieClientConfig(this.conf);
this.hoodieTable = FlinkTables.createTable(writeConfig, hadoopConf, getRuntimeContext());
this.aggregateManager = getRuntimeContext().getGlobalAggregateManager();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -113,7 +113,7 @@ public BucketAssignFunction(Configuration conf) {
@Override
public void open(Configuration parameters) throws Exception {
super.open(parameters);
HoodieWriteConfig writeConfig = StreamerUtil.getHoodieClientConfig(this.conf, false);
HoodieWriteConfig writeConfig = StreamerUtil.getHoodieClientConfig(this.conf);
HoodieFlinkEngineContext context = new HoodieFlinkEngineContext(
new SerializableConfiguration(StreamerUtil.getHadoopConf()),
new FlinkTaskContextSupplier(getRuntimeContext()));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@ public static HoodieFlinkTable<?> createTable(Configuration conf, RuntimeContext
HoodieFlinkEngineContext context = new HoodieFlinkEngineContext(
new SerializableConfiguration(getHadoopConf()),
new FlinkTaskContextSupplier(runtimeContext));
HoodieWriteConfig writeConfig = getHoodieClientConfig(conf, true);
HoodieWriteConfig writeConfig = getHoodieClientConfig(conf);
return HoodieFlinkTable.create(writeConfig, context);
}

Expand All @@ -71,7 +71,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 = StreamerUtil.getHoodieClientConfig(conf);
return HoodieFlinkTable.create(writeConfig, HoodieFlinkEngineContext.DEFAULT);
}
}
52 changes: 4 additions & 48 deletions hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,6 @@
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ReflectionUtils;
import org.apache.hudi.common.util.ValidationUtils;
Expand Down Expand Up @@ -145,21 +144,7 @@ public static org.apache.hadoop.conf.Configuration getHadoopConf() {
return FlinkClientUtil.getHadoopConf();
}

/**
* Mainly used for tests.
*/
public static HoodieWriteConfig getHoodieClientConfig(Configuration conf) {
return getHoodieClientConfig(conf, false, false);
}

public static HoodieWriteConfig getHoodieClientConfig(Configuration conf, boolean loadFsViewStorageConfig) {
return getHoodieClientConfig(conf, false, loadFsViewStorageConfig);
}

public static HoodieWriteConfig getHoodieClientConfig(
Configuration conf,
boolean enableEmbeddedTimelineService,
boolean loadFsViewStorageConfig) {
HoodieWriteConfig.Builder builder =
HoodieWriteConfig.newBuilder()
.withEngineType(EngineType.FLINK)
Expand Down Expand Up @@ -204,20 +189,13 @@ public static HoodieWriteConfig getHoodieClientConfig(
.withPayloadOrderingField(conf.getString(FlinkOptions.PRECOMBINE_FIELD))
.withPayloadEventTimeField(conf.getString(FlinkOptions.PRECOMBINE_FIELD))
.build())
.withEmbeddedTimelineServerEnabled(enableEmbeddedTimelineService)
.withEmbeddedTimelineServerReuseEnabled(true) // make write client embedded timeline service singleton
.withAutoCommit(false)
.withAllowOperationMetadataField(conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED))
.withProps(flinkConf2TypedProperties(conf))
.withSchema(getSourceSchema(conf).toString());

HoodieWriteConfig writeConfig = builder.build();
if (loadFsViewStorageConfig) {
// do not use the builder to give a change for recovering the original fs view storage config
FileSystemViewStorageConfig viewStorageConfig = ViewStorageProperties.loadFromProperties(conf.getString(FlinkOptions.PATH));
writeConfig.setViewStorageConfig(viewStorageConfig);
}
return writeConfig;
return builder.build();
}

/**
Expand Down Expand Up @@ -363,28 +341,15 @@ public static HoodieTableMetaClient createMetaClient(Configuration conf) {

/**
* Creates the Flink write client.
*
* <p>This expects to be used by client, the driver should start an embedded timeline server.
*/
@SuppressWarnings("rawtypes")
public static HoodieFlinkWriteClient createWriteClient(Configuration conf, RuntimeContext runtimeContext) {
return createWriteClient(conf, runtimeContext, true);
}

/**
* Creates the Flink write client.
*
* <p>This expects to be used by client, set flag {@code loadFsViewStorageConfig} to use
* remote filesystem view storage config, or an in-memory filesystem view storage is used.
*/
@SuppressWarnings("rawtypes")
public static HoodieFlinkWriteClient createWriteClient(Configuration conf, RuntimeContext runtimeContext, boolean loadFsViewStorageConfig) {
HoodieFlinkEngineContext context =
new HoodieFlinkEngineContext(
new SerializableConfiguration(getHadoopConf()),
new FlinkTaskContextSupplier(runtimeContext));

HoodieWriteConfig writeConfig = getHoodieClientConfig(conf, loadFsViewStorageConfig);
HoodieWriteConfig writeConfig = getHoodieClientConfig(conf);
return new HoodieFlinkWriteClient<>(context, writeConfig);
}

Expand All @@ -397,18 +362,9 @@ public static HoodieFlinkWriteClient createWriteClient(Configuration conf, Runti
*/
@SuppressWarnings("rawtypes")
public static HoodieFlinkWriteClient createWriteClient(Configuration conf) throws IOException {
HoodieWriteConfig writeConfig = getHoodieClientConfig(conf, true, false);
HoodieWriteConfig writeConfig = getHoodieClientConfig(conf);
// build the write client to start the embedded timeline server
final HoodieFlinkWriteClient writeClient = new HoodieFlinkWriteClient<>(HoodieFlinkEngineContext.DEFAULT, writeConfig);
// create the filesystem view storage properties for client
final FileSystemViewStorageConfig viewStorageConfig = writeConfig.getViewStorageConfig();
// rebuild the view storage config with simplified options.
FileSystemViewStorageConfig rebuilt = FileSystemViewStorageConfig.newBuilder()
.withStorageType(viewStorageConfig.getStorageType())
.withRemoteServerHost(viewStorageConfig.getRemoteViewServerHost())
.withRemoteServerPort(viewStorageConfig.getRemoteViewServerPort()).build();
ViewStorageProperties.createProperties(conf.getString(FlinkOptions.PATH), rebuilt);
return writeClient;
return new HoodieFlinkWriteClient<>(HoodieFlinkEngineContext.DEFAULT, writeConfig);
}

/**
Expand Down

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -19,12 +19,9 @@
package org.apache.hudi.utils;

import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.table.view.FileSystemViewStorageType;
import org.apache.hudi.common.util.FileIOUtils;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.util.StreamerUtil;
import org.apache.hudi.util.ViewStorageProperties;

import org.apache.flink.configuration.Configuration;
import org.junit.jupiter.api.Test;
Expand Down Expand Up @@ -101,13 +98,5 @@ void testInstantTimeDiff() {
long diff = StreamerUtil.instantTimeDiffSeconds(higher, lower);
assertThat(diff, is(75L));
}

@Test
void testDumpRemoteViewStorageConfig() throws IOException {
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
StreamerUtil.createWriteClient(conf);
FileSystemViewStorageConfig storageConfig = ViewStorageProperties.loadFromProperties(conf.getString(FlinkOptions.PATH));
assertThat(storageConfig.getStorageType(), is(FileSystemViewStorageType.REMOTE_FIRST));
}
}

This file was deleted.