[HUDI-3336][HUDI-FLINK]Support custom hadoop config for flink (#5528)
* [HUDI-3336][HUDI-FLINK]Support custom hadoop config for flink
This commit is contained in:
@@ -24,6 +24,7 @@ import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.configuration.HadoopConfigurations;
|
||||
import org.apache.hudi.metadata.HoodieTableMetadata;
|
||||
import org.apache.hudi.sink.event.WriteMetadataEvent;
|
||||
import org.apache.hudi.sink.utils.MockCoordinatorExecutor;
|
||||
@@ -103,7 +104,7 @@ public class TestStreamWriteOperatorCoordinator {
|
||||
|
||||
@Test
|
||||
public void testTableInitialized() throws IOException {
|
||||
final org.apache.hadoop.conf.Configuration hadoopConf = StreamerUtil.getHadoopConf();
|
||||
final org.apache.hadoop.conf.Configuration hadoopConf = HadoopConfigurations.getHadoopConf(new Configuration());
|
||||
String basePath = tempFile.getAbsolutePath();
|
||||
try (FileSystem fs = FSUtils.getFs(basePath, hadoopConf)) {
|
||||
assertTrue(fs.exists(new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME)));
|
||||
@@ -201,7 +202,7 @@ public class TestStreamWriteOperatorCoordinator {
|
||||
assertNotEquals("", instant);
|
||||
|
||||
final String metadataTableBasePath = HoodieTableMetadata.getMetadataTableBasePath(tempFile.getAbsolutePath());
|
||||
HoodieTableMetaClient metadataTableMetaClient = StreamerUtil.createMetaClient(metadataTableBasePath);
|
||||
HoodieTableMetaClient metadataTableMetaClient = StreamerUtil.createMetaClient(metadataTableBasePath, HadoopConfigurations.getHadoopConf(conf));
|
||||
HoodieTimeline completedTimeline = metadataTableMetaClient.getActiveTimeline().filterCompletedInstants();
|
||||
assertThat("One instant need to sync to metadata table", completedTimeline.getInstants().count(), is(1L));
|
||||
assertThat(completedTimeline.lastInstant().get().getTimestamp(), is(HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP));
|
||||
|
||||
@@ -19,6 +19,7 @@
|
||||
package org.apache.hudi.sink.meta;
|
||||
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.configuration.HadoopConfigurations;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
import org.apache.hudi.utils.TestConfigurations;
|
||||
|
||||
@@ -47,7 +48,7 @@ public class TestCkpMetadata {
|
||||
@BeforeEach
|
||||
public void beforeEach() throws Exception {
|
||||
String basePath = tempFile.getAbsolutePath();
|
||||
FileSystem fs = FSUtils.getFs(tempFile.getAbsolutePath(), StreamerUtil.getHadoopConf());
|
||||
FileSystem fs = FSUtils.getFs(tempFile.getAbsolutePath(), HadoopConfigurations.getHadoopConf(new Configuration()));
|
||||
|
||||
Configuration conf = TestConfigurations.getDefaultConf(basePath);
|
||||
StreamerUtil.initTableIfNotExists(conf);
|
||||
|
||||
@@ -24,6 +24,7 @@ import org.apache.hudi.common.config.SerializableConfiguration;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.configuration.HadoopConfigurations;
|
||||
import org.apache.hudi.sink.partitioner.profile.WriteProfile;
|
||||
import org.apache.hudi.table.action.commit.BucketInfo;
|
||||
import org.apache.hudi.table.action.commit.BucketType;
|
||||
@@ -71,7 +72,7 @@ public class TestBucketAssigner {
|
||||
|
||||
writeConfig = StreamerUtil.getHoodieClientConfig(conf);
|
||||
context = new HoodieFlinkEngineContext(
|
||||
new SerializableConfiguration(StreamerUtil.getHadoopConf()),
|
||||
new SerializableConfiguration(HadoopConfigurations.getHadoopConf(conf)),
|
||||
new FlinkTaskContextSupplier(null));
|
||||
StreamerUtil.initTableIfNotExists(conf);
|
||||
}
|
||||
|
||||
@@ -26,6 +26,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.TableSchemaResolver;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.configuration.HadoopConfigurations;
|
||||
import org.apache.hudi.configuration.OptionsResolver;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.sink.event.WriteMetadataEvent;
|
||||
@@ -345,7 +346,7 @@ public class TestWriteBase {
|
||||
}
|
||||
|
||||
private void checkWrittenDataMor(File baseFile, Map<String, String> expected, int partitions) throws Exception {
|
||||
HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(basePath);
|
||||
HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(basePath, HadoopConfigurations.getHadoopConf(conf));
|
||||
Schema schema = new TableSchemaResolver(metaClient).getTableAvroSchema();
|
||||
String latestInstant = lastCompleteInstant();
|
||||
FileSystem fs = FSUtils.getFs(basePath, new org.apache.hadoop.conf.Configuration());
|
||||
|
||||
@@ -21,6 +21,7 @@ package org.apache.hudi.source;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.TableSchemaResolver;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.configuration.HadoopConfigurations;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.table.format.mor.MergeOnReadInputFormat;
|
||||
import org.apache.hudi.table.format.mor.MergeOnReadInputSplit;
|
||||
@@ -239,7 +240,7 @@ public class TestStreamReadOperator {
|
||||
|
||||
private OneInputStreamOperatorTestHarness<MergeOnReadInputSplit, RowData> createReader() throws Exception {
|
||||
final String basePath = tempFile.getAbsolutePath();
|
||||
final org.apache.hadoop.conf.Configuration hadoopConf = StreamerUtil.getHadoopConf();
|
||||
final org.apache.hadoop.conf.Configuration hadoopConf = HadoopConfigurations.getHadoopConf(new Configuration());
|
||||
final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
|
||||
.setConf(hadoopConf).setBasePath(basePath).build();
|
||||
final List<String> partitionKeys = Collections.singletonList("partition");
|
||||
|
||||
@@ -22,6 +22,7 @@ import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.configuration.HadoopConfigurations;
|
||||
import org.apache.hudi.table.HoodieTableSource;
|
||||
import org.apache.hudi.table.format.cow.CopyOnWriteInputFormat;
|
||||
import org.apache.hudi.table.format.mor.MergeOnReadInputFormat;
|
||||
@@ -400,7 +401,7 @@ public class TestInputFormat {
|
||||
TestData.writeData(dataset, conf);
|
||||
}
|
||||
|
||||
HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(tempFile.getAbsolutePath());
|
||||
HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(tempFile.getAbsolutePath(), HadoopConfigurations.getHadoopConf(conf));
|
||||
List<String> commits = metaClient.getCommitsTimeline().filterCompletedInstants().getInstants()
|
||||
.map(HoodieInstant::getTimestamp).collect(Collectors.toList());
|
||||
|
||||
|
||||
@@ -106,7 +106,7 @@ public class TestStreamerUtil {
|
||||
void testDumpRemoteViewStorageConfig() throws IOException {
|
||||
Configuration conf = TestConfigurations.getDefaultConf(tempFile.getAbsolutePath());
|
||||
StreamerUtil.createWriteClient(conf);
|
||||
FileSystemViewStorageConfig storageConfig = ViewStorageProperties.loadFromProperties(conf.getString(FlinkOptions.PATH));
|
||||
FileSystemViewStorageConfig storageConfig = ViewStorageProperties.loadFromProperties(conf.getString(FlinkOptions.PATH), new Configuration());
|
||||
assertThat(storageConfig.getStorageType(), is(FileSystemViewStorageType.REMOTE_FIRST));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -22,6 +22,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.configuration.FlinkOptions;
|
||||
import org.apache.hudi.configuration.HadoopConfigurations;
|
||||
import org.apache.hudi.source.StreamReadMonitoringFunction;
|
||||
import org.apache.hudi.table.format.mor.MergeOnReadInputSplit;
|
||||
import org.apache.hudi.util.StreamerUtil;
|
||||
@@ -39,19 +40,19 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
public class TestUtils {
|
||||
public static String getLastPendingInstant(String basePath) {
|
||||
final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
|
||||
.setConf(StreamerUtil.getHadoopConf()).setBasePath(basePath).build();
|
||||
.setConf(HadoopConfigurations.getHadoopConf(new Configuration())).setBasePath(basePath).build();
|
||||
return StreamerUtil.getLastPendingInstant(metaClient);
|
||||
}
|
||||
|
||||
public static String getLastCompleteInstant(String basePath) {
|
||||
final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
|
||||
.setConf(StreamerUtil.getHadoopConf()).setBasePath(basePath).build();
|
||||
.setConf(HadoopConfigurations.getHadoopConf(new Configuration())).setBasePath(basePath).build();
|
||||
return StreamerUtil.getLastCompletedInstant(metaClient);
|
||||
}
|
||||
|
||||
public static String getLastDeltaCompleteInstant(String basePath) {
|
||||
final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
|
||||
.setConf(StreamerUtil.getHadoopConf()).setBasePath(basePath).build();
|
||||
.setConf(HadoopConfigurations.getHadoopConf(new Configuration())).setBasePath(basePath).build();
|
||||
return metaClient.getCommitsTimeline().filterCompletedInstants()
|
||||
.filter(hoodieInstant -> hoodieInstant.getAction().equals(HoodieTimeline.DELTA_COMMIT_ACTION))
|
||||
.lastInstant()
|
||||
@@ -61,7 +62,7 @@ public class TestUtils {
|
||||
|
||||
public static String getFirstCompleteInstant(String basePath) {
|
||||
final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
|
||||
.setConf(StreamerUtil.getHadoopConf()).setBasePath(basePath).build();
|
||||
.setConf(HadoopConfigurations.getHadoopConf(new Configuration())).setBasePath(basePath).build();
|
||||
return metaClient.getCommitsAndCompactionTimeline().filterCompletedInstants().firstInstant()
|
||||
.map(HoodieInstant::getTimestamp).orElse(null);
|
||||
}
|
||||
@@ -69,7 +70,7 @@ public class TestUtils {
|
||||
@Nullable
|
||||
public static String getNthCompleteInstant(String basePath, int n, boolean isDelta) {
|
||||
final HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
|
||||
.setConf(StreamerUtil.getHadoopConf()).setBasePath(basePath).build();
|
||||
.setConf(HadoopConfigurations.getHadoopConf(new Configuration())).setBasePath(basePath).build();
|
||||
return metaClient.getActiveTimeline()
|
||||
.filterCompletedInstants()
|
||||
.filter(instant -> isDelta ? HoodieTimeline.DELTA_COMMIT_ACTION.equals(instant.getAction()) : HoodieTimeline.COMMIT_ACTION.equals(instant.getAction()))
|
||||
|
||||
@@ -18,6 +18,7 @@
|
||||
|
||||
package org.apache.hudi.utils;
|
||||
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
|
||||
import org.apache.hudi.common.table.view.FileSystemViewStorageType;
|
||||
import org.apache.hudi.util.ViewStorageProperties;
|
||||
@@ -45,11 +46,12 @@ public class TestViewStorageProperties {
|
||||
.withStorageType(FileSystemViewStorageType.SPILLABLE_DISK)
|
||||
.withRemoteServerHost("host1")
|
||||
.withRemoteServerPort(1234).build();
|
||||
ViewStorageProperties.createProperties(basePath, config);
|
||||
ViewStorageProperties.createProperties(basePath, config);
|
||||
ViewStorageProperties.createProperties(basePath, config);
|
||||
Configuration flinkConfig = new Configuration();
|
||||
ViewStorageProperties.createProperties(basePath, config, flinkConfig);
|
||||
ViewStorageProperties.createProperties(basePath, config, flinkConfig);
|
||||
ViewStorageProperties.createProperties(basePath, config, flinkConfig);
|
||||
|
||||
FileSystemViewStorageConfig readConfig = ViewStorageProperties.loadFromProperties(basePath);
|
||||
FileSystemViewStorageConfig readConfig = ViewStorageProperties.loadFromProperties(basePath, new Configuration());
|
||||
assertThat(readConfig.getStorageType(), is(FileSystemViewStorageType.SPILLABLE_DISK));
|
||||
assertThat(readConfig.getRemoteViewServerHost(), is("host1"));
|
||||
assertThat(readConfig.getRemoteViewServerPort(), is(1234));
|
||||
|
||||
Reference in New Issue
Block a user