1
0

[HUDI-2537] Fix metadata table for flink (#3774)

This commit is contained in:
Danny Chan
2021-10-10 09:30:39 +08:00
committed by GitHub
parent 2a392d8e8e
commit ad63938890
12 changed files with 231 additions and 152 deletions

View File

@@ -126,11 +126,6 @@ public class StreamWriteOperatorCoordinator
*/
private HiveSyncContext hiveSyncContext;
/**
* A single-thread executor to handle metadata table sync.
*/
private NonThrownExecutor metadataSyncExecutor;
/**
* The table state.
*/
@@ -294,7 +289,7 @@ public class StreamWriteOperatorCoordinator
}
private void initMetadataSync() {
this.metadataSyncExecutor = new NonThrownExecutor(LOG, true);
this.writeClient.initMetadataWriter();
}
private void reset() {
@@ -498,14 +493,6 @@ public class StreamWriteOperatorCoordinator
this.executor = executor;
}
@VisibleForTesting
public void setMetadataSyncExecutor(NonThrownExecutor executor) throws Exception {
if (this.metadataSyncExecutor != null) {
this.metadataSyncExecutor.close();
}
this.metadataSyncExecutor = executor;
}
// -------------------------------------------------------------------------
// Inner Class
// -------------------------------------------------------------------------

View File

@@ -72,8 +72,6 @@ public class AppendWriteFunction<I> extends AbstractStreamWriteFunction<I> {
// it would check the validity.
// wait for the buffer data flush out and request a new instant
flushData(false);
// nullify the write helper for next ckp
this.writerHelper = null;
}
@Override
@@ -133,5 +131,10 @@ public class AppendWriteFunction<I> extends AbstractStreamWriteFunction<I> {
.endInput(endInput)
.build();
this.eventGateway.sendEventToCoordinator(event);
// nullify the write helper for next ckp
this.writerHelper = null;
this.writeStatuses.addAll(writeStatus);
// blocks flushing until the coordinator starts a new instant
this.confirming = true;
}
}

View File

@@ -22,6 +22,7 @@ import org.apache.hudi.client.common.HoodieFlinkEngineContext;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.util.StreamerUtil;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.configuration.Configuration;
@@ -48,10 +49,12 @@ public class FileIndex {
private final Path path;
private final HoodieMetadataConfig metadataConfig;
private List<String> partitionPaths; // cache of partition paths
private final boolean tableExists;
private FileIndex(Path path, Configuration conf) {
this.path = path;
this.metadataConfig = metadataConfig(conf);
this.tableExists = StreamerUtil.tableExists(path.toString(), StreamerUtil.getHadoopConf());
}
public static FileIndex instance(Path path, Configuration conf) {
@@ -111,6 +114,9 @@ public class FileIndex {
* Returns all the file statuses under the table base path.
*/
public FileStatus[] getFilesInPartitions() {
if (!tableExists) {
return new FileStatus[0];
}
String[] partitions = getOrBuildPartitionPaths().stream().map(p -> fullPartitionPath(path, p)).toArray(String[]::new);
return FSUtils.getFilesInPartitions(HoodieFlinkEngineContext.DEFAULT, metadataConfig, path.toString(),
partitions, "/tmp/")
@@ -165,8 +171,9 @@ public class FileIndex {
if (this.partitionPaths != null) {
return this.partitionPaths;
}
this.partitionPaths = FSUtils.getAllPartitionPaths(HoodieFlinkEngineContext.DEFAULT,
metadataConfig, path.toString());
this.partitionPaths = this.tableExists
? FSUtils.getAllPartitionPaths(HoodieFlinkEngineContext.DEFAULT, metadataConfig, path.toString())
: Collections.emptyList();
return this.partitionPaths;
}
@@ -174,7 +181,7 @@ public class FileIndex {
Properties properties = new Properties();
// set up metadata.enabled=true in table DDL to enable metadata listing
properties.put(HoodieMetadataConfig.ENABLE, conf.getBoolean(FlinkOptions.METADATA_ENABLED));
properties.put(HoodieMetadataConfig.ENABLE.key(), conf.getBoolean(FlinkOptions.METADATA_ENABLED));
return HoodieMetadataConfig.newBuilder().fromProperties(properties).build();
}