1
0

[HUDI-1681] Support object storage for Flink writer (#2662)

In order to support object storage, we need these changes:

* Use the Hadoop filesystem so that we can find the plugin filesystem
* Do not fetch file size until the file handle is closed
* Do not close the opened filesystem because we want to use the
  filesystem cache
This commit is contained in:
Danny Chan
2021-03-12 16:39:24 +08:00
committed by GitHub
parent e8e6708aea
commit 20786ab8a2
18 changed files with 443 additions and 164 deletions

View File

@@ -28,8 +28,8 @@ import org.apache.hudi.util.StreamerUtil;
import org.apache.flink.api.common.io.FileInputFormat;
import org.apache.flink.api.common.io.InputFormat;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.fs.Path;
import org.apache.flink.table.data.RowData;
import org.apache.hadoop.fs.Path;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;

View File

@@ -36,7 +36,6 @@ import org.apache.hudi.utils.TestUtils;
import org.apache.avro.Schema;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.fs.Path;
import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
import org.apache.flink.streaming.api.TimeCharacteristic;
import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory;
@@ -48,6 +47,7 @@ import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.logical.RowType;
import org.apache.hadoop.fs.Path;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
@@ -242,8 +242,9 @@ 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 HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
.setConf(StreamerUtil.getHadoopConf()).setBasePath(basePath).build();
.setConf(hadoopConf).setBasePath(basePath).build();
final List<String> partitionKeys = Collections.singletonList("partition");
// This input format is used to opening the emitted split.
@@ -262,11 +263,10 @@ public class TestStreamReadOperator {
tableAvroSchema.toString(),
AvroSchemaConverter.convertToSchema(TestConfigurations.ROW_TYPE).toString(),
Collections.emptyList());
Path[] paths = FilePathUtils.getReadPaths(
new Path(basePath), conf, partitionKeys, conf.getString(FlinkOptions.PARTITION_DEFAULT_NAME));
Path[] paths = FilePathUtils.getReadPaths(new Path(basePath), conf, hadoopConf, partitionKeys);
MergeOnReadInputFormat inputFormat = new MergeOnReadInputFormat(
conf,
paths,
FilePathUtils.toFlinkPaths(paths),
hoodieTableState,
rowDataType.getChildren(),
"default",

View File

@@ -27,9 +27,9 @@ import org.apache.hudi.util.StreamerUtil;
import org.apache.flink.api.common.io.InputFormat;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.fs.Path;
import org.apache.flink.core.io.InputSplit;
import org.apache.flink.table.data.RowData;
import org.apache.hadoop.fs.Path;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
import org.junit.jupiter.params.ParameterizedTest;