3333import org .apache .hudi .common .table .timeline .HoodieActiveTimeline ;
3434import org .apache .hudi .common .table .timeline .HoodieInstant ;
3535import org .apache .hudi .common .table .timeline .HoodieTimeline ;
36- import org .apache .hudi .common .table .view .FileSystemViewStorageConfig ;
3736import org .apache .hudi .common .util .Option ;
3837import org .apache .hudi .common .util .ReflectionUtils ;
3938import org .apache .hudi .common .util .ValidationUtils ;
@@ -145,21 +144,7 @@ public static org.apache.hadoop.conf.Configuration getHadoopConf() {
145144 return FlinkClientUtil .getHadoopConf ();
146145 }
147146
148- /**
149- * Mainly used for tests.
150- */
151147 public static HoodieWriteConfig getHoodieClientConfig (Configuration conf ) {
152- return getHoodieClientConfig (conf , false , false );
153- }
154-
155- public static HoodieWriteConfig getHoodieClientConfig (Configuration conf , boolean loadFsViewStorageConfig ) {
156- return getHoodieClientConfig (conf , false , loadFsViewStorageConfig );
157- }
158-
159- public static HoodieWriteConfig getHoodieClientConfig (
160- Configuration conf ,
161- boolean enableEmbeddedTimelineService ,
162- boolean loadFsViewStorageConfig ) {
163148 HoodieWriteConfig .Builder builder =
164149 HoodieWriteConfig .newBuilder ()
165150 .withEngineType (EngineType .FLINK )
@@ -204,20 +189,13 @@ public static HoodieWriteConfig getHoodieClientConfig(
204189 .withPayloadOrderingField (conf .getString (FlinkOptions .PRECOMBINE_FIELD ))
205190 .withPayloadEventTimeField (conf .getString (FlinkOptions .PRECOMBINE_FIELD ))
206191 .build ())
207- .withEmbeddedTimelineServerEnabled (enableEmbeddedTimelineService )
208192 .withEmbeddedTimelineServerReuseEnabled (true ) // make write client embedded timeline service singleton
209193 .withAutoCommit (false )
210194 .withAllowOperationMetadataField (conf .getBoolean (FlinkOptions .CHANGELOG_ENABLED ))
211195 .withProps (flinkConf2TypedProperties (conf ))
212196 .withSchema (getSourceSchema (conf ).toString ());
213197
214- HoodieWriteConfig writeConfig = builder .build ();
215- if (loadFsViewStorageConfig ) {
216- // do not use the builder to give a change for recovering the original fs view storage config
217- FileSystemViewStorageConfig viewStorageConfig = ViewStorageProperties .loadFromProperties (conf .getString (FlinkOptions .PATH ));
218- writeConfig .setViewStorageConfig (viewStorageConfig );
219- }
220- return writeConfig ;
198+ return builder .build ();
221199 }
222200
223201 /**
@@ -363,28 +341,15 @@ public static HoodieTableMetaClient createMetaClient(Configuration conf) {
363341
364342 /**
365343 * Creates the Flink write client.
366- *
367- * <p>This expects to be used by client, the driver should start an embedded timeline server.
368344 */
369345 @ SuppressWarnings ("rawtypes" )
370346 public static HoodieFlinkWriteClient createWriteClient (Configuration conf , RuntimeContext runtimeContext ) {
371- return createWriteClient (conf , runtimeContext , true );
372- }
373-
374- /**
375- * Creates the Flink write client.
376- *
377- * <p>This expects to be used by client, set flag {@code loadFsViewStorageConfig} to use
378- * remote filesystem view storage config, or an in-memory filesystem view storage is used.
379- */
380- @ SuppressWarnings ("rawtypes" )
381- public static HoodieFlinkWriteClient createWriteClient (Configuration conf , RuntimeContext runtimeContext , boolean loadFsViewStorageConfig ) {
382347 HoodieFlinkEngineContext context =
383348 new HoodieFlinkEngineContext (
384349 new SerializableConfiguration (getHadoopConf ()),
385350 new FlinkTaskContextSupplier (runtimeContext ));
386351
387- HoodieWriteConfig writeConfig = getHoodieClientConfig (conf , loadFsViewStorageConfig );
352+ HoodieWriteConfig writeConfig = getHoodieClientConfig (conf );
388353 return new HoodieFlinkWriteClient <>(context , writeConfig );
389354 }
390355
@@ -397,18 +362,9 @@ public static HoodieFlinkWriteClient createWriteClient(Configuration conf, Runti
397362 */
398363 @ SuppressWarnings ("rawtypes" )
399364 public static HoodieFlinkWriteClient createWriteClient (Configuration conf ) throws IOException {
400- HoodieWriteConfig writeConfig = getHoodieClientConfig (conf , true , false );
365+ HoodieWriteConfig writeConfig = getHoodieClientConfig (conf );
401366 // build the write client to start the embedded timeline server
402- final HoodieFlinkWriteClient writeClient = new HoodieFlinkWriteClient <>(HoodieFlinkEngineContext .DEFAULT , writeConfig );
403- // create the filesystem view storage properties for client
404- final FileSystemViewStorageConfig viewStorageConfig = writeConfig .getViewStorageConfig ();
405- // rebuild the view storage config with simplified options.
406- FileSystemViewStorageConfig rebuilt = FileSystemViewStorageConfig .newBuilder ()
407- .withStorageType (viewStorageConfig .getStorageType ())
408- .withRemoteServerHost (viewStorageConfig .getRemoteViewServerHost ())
409- .withRemoteServerPort (viewStorageConfig .getRemoteViewServerPort ()).build ();
410- ViewStorageProperties .createProperties (conf .getString (FlinkOptions .PATH ), rebuilt );
411- return writeClient ;
367+ return new HoodieFlinkWriteClient <>(HoodieFlinkEngineContext .DEFAULT , writeConfig );
412368 }
413369
414370 /**
0 commit comments