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

@@ -274,6 +274,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
if (!stat.getLogFiles().contains(result.logFile().getFileName())) {
stat.addLogFiles(result.logFile().getFileName());
}
stat.setFileSizeInBytes(result.size());
}
private void updateRuntimeStats(HoodieDeltaWriteStat stat) {
@@ -304,6 +305,7 @@ public class HoodieAppendHandle<T extends HoodieRecordPayload, I, K, O> extends
} else if (stat.getPath().endsWith(result.logFile().getFileName())) {
// append/continued writing to the same log file
stat.setLogOffset(Math.min(stat.getLogOffset(), result.offset()));
stat.setFileSizeInBytes(stat.getFileSizeInBytes() + result.size());
accumulateWriteCounts(stat, result);
accumulateRuntimeStats(stat);
} else {

View File

@@ -33,4 +33,6 @@ public interface HoodieFileWriter<R extends IndexedRecord> {
void close() throws IOException;
void writeAvro(String key, R oldRecord) throws IOException;
long getBytesWritten();
}

View File

@@ -156,4 +156,9 @@ public class HoodieHFileWriter<T extends HoodieRecordPayload, R extends IndexedR
writer.close();
writer = null;
}
@Override
public long getBytesWritten() {
return fs.getBytesWritten(file);
}
}

View File

@@ -94,4 +94,9 @@ public class HoodieParquetWriter<T extends HoodieRecordPayload, R extends Indexe
super.write(object);
writeSupport.add(key);
}
@Override
public long getBytesWritten() {
return fs.getBytesWritten(file);
}
}

View File

@@ -46,6 +46,12 @@
<groupId>org.apache.flink</groupId>
<artifactId>flink-clients_${scala.binary.version}</artifactId>
</dependency>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-hadoop-compatibility_${scala.binary.version}</artifactId>
<version>${flink.version}</version>
<scope>provided</scope>
</dependency>
<!-- Parquet -->
<dependency>

View File

@@ -29,7 +29,6 @@ import org.apache.hudi.common.function.SerializablePairFunction;
import org.apache.hudi.common.util.Option;
import org.apache.flink.api.common.functions.RuntimeContext;
import org.apache.hadoop.conf.Configuration;
import java.util.List;
import java.util.Map;
@@ -37,6 +36,7 @@ import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.util.FlinkClientUtil;
import static org.apache.hudi.common.function.FunctionWrapper.throwingFlatMapWrapper;
import static org.apache.hudi.common.function.FunctionWrapper.throwingForeachWrapper;
@@ -50,7 +50,7 @@ public class HoodieFlinkEngineContext extends HoodieEngineContext {
private RuntimeContext runtimeContext;
public HoodieFlinkEngineContext(TaskContextSupplier taskContextSupplier) {
this(new SerializableConfiguration(new Configuration()), taskContextSupplier);
this(new SerializableConfiguration(FlinkClientUtil.getHadoopConf()), taskContextSupplier);
}
public HoodieFlinkEngineContext(SerializableConfiguration hadoopConf, TaskContextSupplier taskContextSupplier) {

View File

@@ -20,14 +20,12 @@ package org.apache.hudi.io;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.engine.TaskContextSupplier;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieUpsertException;
import org.apache.hudi.table.HoodieTable;
import org.apache.hadoop.fs.Path;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -98,14 +96,7 @@ public class FlinkAppendHandle<T extends HoodieRecordPayload, I, K, O> extends H
needBootStrap = false;
// flush any remaining records to disk
appendDataAndDeleteBlocks(header);
try {
for (WriteStatus status: statuses) {
long logFileSize = FSUtils.getFileSize(fs, new Path(config.getBasePath(), status.getStat().getPath()));
status.getStat().setFileSizeInBytes(logFileSize);
}
} catch (IOException e) {
throw new HoodieUpsertException("Failed to get file size for append handle", e);
}
// need to fix that the incremental write size in bytes may be lost
List<WriteStatus> ret = new ArrayList<>(statuses);
statuses.clear();
return ret;

View File

@@ -20,7 +20,6 @@ package org.apache.hudi.io;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.engine.TaskContextSupplier;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.HoodieWriteStat;
@@ -109,7 +108,7 @@ public class FlinkCreateHandle<T extends HoodieRecordPayload, I, K, O>
* @throws IOException if error occurs
*/
private void setUpWriteStatus() throws IOException {
long fileSizeInBytes = FSUtils.getFileSize(fs, path);
long fileSizeInBytes = fileWriter.getBytesWritten();
long incFileSizeInBytes = fileSizeInBytes - lastFileSize;
this.lastFileSize = fileSizeInBytes;
HoodieWriteStat stat = new HoodieWriteStat();

View File

@@ -0,0 +1,81 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.util;
import org.apache.flink.api.java.hadoop.mapred.utils.HadoopUtils;
import org.apache.flink.configuration.Configuration;
import org.apache.hadoop.fs.Path;
import java.io.File;
/**
* Utilities for Hoodie Flink client.
*/
public class FlinkClientUtil {
/**
* Returns the hadoop configuration with possible hadoop conf paths.
* E.G. the configurations under path $HADOOP_CONF_DIR and $HADOOP_HOME.
*/
public static org.apache.hadoop.conf.Configuration getHadoopConf() {
// create hadoop configuration with hadoop conf directory configured.
org.apache.hadoop.conf.Configuration hadoopConf = null;
for (String possibleHadoopConfPath : HadoopUtils.possibleHadoopConfPaths(new Configuration())) {
hadoopConf = getHadoopConfiguration(possibleHadoopConfPath);
if (hadoopConf != null) {
break;
}
}
if (hadoopConf == null) {
hadoopConf = new org.apache.hadoop.conf.Configuration();
}
return hadoopConf;
}
/**
* Returns a new Hadoop Configuration object using the path to the hadoop conf configured.
*
* @param hadoopConfDir Hadoop conf directory path.
* @return A Hadoop configuration instance.
*/
private static org.apache.hadoop.conf.Configuration getHadoopConfiguration(String hadoopConfDir) {
if (new File(hadoopConfDir).exists()) {
org.apache.hadoop.conf.Configuration hadoopConfiguration = new org.apache.hadoop.conf.Configuration();
File coreSite = new File(hadoopConfDir, "core-site.xml");
if (coreSite.exists()) {
hadoopConfiguration.addResource(new Path(coreSite.getAbsolutePath()));
}
File hdfsSite = new File(hadoopConfDir, "hdfs-site.xml");
if (hdfsSite.exists()) {
hadoopConfiguration.addResource(new Path(hdfsSite.getAbsolutePath()));
}
File yarnSite = new File(hadoopConfDir, "yarn-site.xml");
if (yarnSite.exists()) {
hadoopConfiguration.addResource(new Path(yarnSite.getAbsolutePath()));
}
// Add mapred-site.xml. We need to read configurations like compression codec.
File mapredSite = new File(hadoopConfDir, "mapred-site.xml");
if (mapredSite.exists()) {
hadoopConfiguration.addResource(new Path(mapredSite.getAbsolutePath()));
}
return hadoopConfiguration;
}
return null;
}
}