1
0

[HUDI-2562] Embedded timeline server on JobManager (#3812)

This commit is contained in:
Danny Chan
2021-10-18 10:45:39 +08:00
committed by GitHub
parent 9aa7cfb802
commit 2eda3de7f9
34 changed files with 540 additions and 334 deletions

View File

@@ -139,7 +139,7 @@ public class StreamWriteFunction<I> extends AbstractStreamWriteFunction<I> {
public void close() {
if (this.writeClient != null) {
this.writeClient.cleanHandlesGracefully();
// this.writeClient.close();
this.writeClient.close();
}
}
@@ -155,7 +155,6 @@ public class StreamWriteFunction<I> extends AbstractStreamWriteFunction<I> {
// -------------------------------------------------------------------------
// Getter/Setter
// -------------------------------------------------------------------------
@VisibleForTesting
@SuppressWarnings("rawtypes")
public Map<String, List<HoodieRecord>> getDataBuffer() {

View File

@@ -22,6 +22,7 @@ import org.apache.hudi.client.HoodieFlinkWriteClient;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.CommitUtils;
@@ -41,6 +42,7 @@ import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.jobgraph.OperatorID;
import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
import org.apache.flink.runtime.operators.coordination.OperatorEvent;
import org.apache.flink.runtime.operators.coordination.TaskNotRunningException;
import org.jetbrains.annotations.Nullable;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -57,7 +59,6 @@ import java.util.Objects;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import static org.apache.hudi.util.StreamerUtil.initTableIfNotExists;
@@ -95,6 +96,11 @@ public class StreamWriteOperatorCoordinator
*/
private transient HoodieFlinkWriteClient writeClient;
/**
* Meta client.
*/
private transient HoodieTableMetaClient metaClient;
/**
* Current REQUESTED instant, for validation.
*/
@@ -153,10 +159,11 @@ public class StreamWriteOperatorCoordinator
// initialize event buffer
reset();
this.gateways = new SubtaskGateway[this.parallelism];
// init table, create if not exists.
this.metaClient = initTableIfNotExists(this.conf);
// the write client must create after the table creation
this.writeClient = StreamerUtil.createWriteClient(conf);
this.tableState = TableState.create(conf);
// init table, create it if not exists.
initTableIfNotExists(this.conf);
// start the executor
this.executor = new CoordinatorExecutor(this.context, LOG);
// start the executor if required
@@ -171,15 +178,17 @@ public class StreamWriteOperatorCoordinator
@Override
public void close() throws Exception {
// teardown the resource
if (writeClient != null) {
writeClient.close();
}
if (executor != null) {
executor.close();
}
if (hiveSyncExecutor != null) {
hiveSyncExecutor.close();
}
// the write client must close after the executor service
// because the task in the service may send requests to the embedded timeline service.
if (writeClient != null) {
writeClient.close();
}
this.eventBuffer = null;
}
@@ -225,6 +234,14 @@ public class StreamWriteOperatorCoordinator
);
}
@Override
public void notifyCheckpointAborted(long checkpointId) {
// once the checkpoint was aborted, unblock the writer tasks to
// reuse the last instant.
executor.execute(this::sendCommitAckEvents,
"unblock data write with aborted checkpoint %s", checkpointId);
}
@Override
public void resetToCheckpoint(long checkpointID, byte[] checkpointData) {
// no operation
@@ -316,7 +333,7 @@ public class StreamWriteOperatorCoordinator
final String instant = HoodieActiveTimeline.createNewInstantTime();
this.writeClient.startCommitWithTime(instant, tableState.commitAction);
this.instant = instant;
this.writeClient.transitionRequestedToInflight(tableState.commitAction, this.instant);
this.metaClient.getActiveTimeline().transitionRequestedToInflight(tableState.commitAction, this.instant);
this.writeClient.upgradeDowngrade(this.instant);
LOG.info("Create instant [{}] for table [{}] with type [{}]", this.instant,
this.conf.getString(FlinkOptions.TABLE_NAME), conf.getString(FlinkOptions.TABLE_TYPE));
@@ -382,16 +399,28 @@ public class StreamWriteOperatorCoordinator
* sends the commit ack events to unblock the flushing.
*/
private void sendCommitAckEvents() {
CompletableFuture<?>[] futures = IntStream.range(0, this.parallelism)
.mapToObj(taskID -> this.gateways[taskID].sendEvent(CommitAckEvent.getInstance()))
CompletableFuture<?>[] futures = Arrays.stream(this.gateways).filter(Objects::nonNull)
.map(gw -> gw.sendEvent(CommitAckEvent.getInstance()))
.toArray(CompletableFuture<?>[]::new);
try {
CompletableFuture.allOf(futures).get();
} catch (Exception e) {
throw new HoodieException("Error while waiting for the commit ack events to finish sending", e);
} catch (Throwable throwable) {
if (!sendToFinishedTasks(throwable)) {
throw new HoodieException("Error while waiting for the commit ack events to finish sending", throwable);
}
}
}
/**
* Decides whether the given exception is caused by sending events to FINISHED tasks.
*
* <p>Ugly impl: the exception may change in the future.
*/
private static boolean sendToFinishedTasks(Throwable throwable) {
return throwable.getCause() instanceof TaskNotRunningException
|| throwable.getCause().getMessage().contains("running");
}
/**
* Commits the instant.
*
@@ -474,12 +503,6 @@ public class StreamWriteOperatorCoordinator
return instant;
}
@VisibleForTesting
@SuppressWarnings("rawtypes")
public HoodieFlinkWriteClient getWriteClient() {
return writeClient;
}
@VisibleForTesting
public Context getContext() {
return context;

View File

@@ -82,20 +82,11 @@ public class AppendWriteFunction<I> extends AbstractStreamWriteFunction<I> {
this.writerHelper.write((RowData) value);
}
@Override
public void close() {
if (this.writeClient != null) {
this.writeClient.cleanHandlesGracefully();
this.writeClient.close();
}
}
/**
* End input action for batch source.
*/
public void endInput() {
flushData(true);
this.writeClient.cleanHandles();
this.writeStatuses.clear();
}

View File

@@ -19,7 +19,6 @@
package org.apache.hudi.sink.bootstrap;
import org.apache.hudi.client.FlinkTaskContextSupplier;
import org.apache.hudi.client.HoodieFlinkWriteClient;
import org.apache.hudi.client.common.HoodieFlinkEngineContext;
import org.apache.hudi.common.config.SerializableConfiguration;
import org.apache.hudi.common.fs.FSUtils;
@@ -27,14 +26,12 @@ import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordGlobalLocation;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.TableSchemaResolver;
import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.BaseFileUtils;
import org.apache.hudi.common.util.CommitUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.config.HoodieWriteConfig;
@@ -92,8 +89,6 @@ public class BootstrapOperator<I, O extends HoodieRecord>
private transient ListState<String> instantState;
private final Pattern pattern;
private String lastInstantTime;
private HoodieFlinkWriteClient writeClient;
private String actionType;
public BootstrapOperator(Configuration conf) {
this.conf = conf;
@@ -102,7 +97,8 @@ public class BootstrapOperator<I, O extends HoodieRecord>
@Override
public void snapshotState(StateSnapshotContext context) throws Exception {
lastInstantTime = this.writeClient.getLastPendingInstant(this.actionType);
HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(this.conf);
lastInstantTime = StreamerUtil.getLastPendingInstant(metaClient);
instantState.update(Collections.singletonList(lastInstantTime));
}
@@ -122,12 +118,8 @@ public class BootstrapOperator<I, O extends HoodieRecord>
}
this.hadoopConf = StreamerUtil.getHadoopConf();
this.writeConfig = StreamerUtil.getHoodieClientConfig(this.conf);
this.writeConfig = StreamerUtil.getHoodieClientConfig(this.conf, true);
this.hoodieTable = getTable();
this.writeClient = StreamerUtil.createWriteClient(this.conf, getRuntimeContext());
this.actionType = CommitUtils.getCommitActionType(
WriteOperationType.fromValue(conf.getString(FlinkOptions.OPERATION)),
HoodieTableType.valueOf(conf.getString(FlinkOptions.TABLE_TYPE)));
preLoadIndexRecords();
}

View File

@@ -20,9 +20,8 @@ package org.apache.hudi.sink.bulk;
import org.apache.hudi.client.HoodieFlinkWriteClient;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.util.CommitUtils;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.sink.StreamWriteOperatorCoordinator;
import org.apache.hudi.sink.common.AbstractWriteFunction;
@@ -80,6 +79,11 @@ public class BulkInsertWriteFunction<I>
*/
private int taskID;
/**
* Meta Client.
*/
private transient HoodieTableMetaClient metaClient;
/**
* Write Client.
*/
@@ -95,11 +99,6 @@ public class BulkInsertWriteFunction<I>
*/
private transient OperatorEventGateway eventGateway;
/**
* Commit action type.
*/
private transient String actionType;
/**
* Constructs a StreamingSinkFunction.
*
@@ -113,12 +112,9 @@ public class BulkInsertWriteFunction<I>
@Override
public void open(Configuration parameters) throws IOException {
this.taskID = getRuntimeContext().getIndexOfThisSubtask();
this.metaClient = StreamerUtil.createMetaClient(this.config);
this.writeClient = StreamerUtil.createWriteClient(this.config, getRuntimeContext());
this.actionType = CommitUtils.getCommitActionType(
WriteOperationType.fromValue(config.getString(FlinkOptions.OPERATION)),
HoodieTableType.valueOf(config.getString(FlinkOptions.TABLE_TYPE)));
this.initInstant = this.writeClient.getLastPendingInstant(this.actionType);
this.initInstant = StreamerUtil.getLastPendingInstant(this.metaClient, false);
sendBootstrapEvent();
initWriterHelper();
}
@@ -188,12 +184,13 @@ public class BulkInsertWriteFunction<I>
}
private String instantToWrite() {
String instant = this.writeClient.getLastPendingInstant(this.actionType);
String instant = StreamerUtil.getLastPendingInstant(this.metaClient);
// if exactly-once semantics turns on,
// waits for the checkpoint notification until the checkpoint timeout threshold hits.
TimeWait timeWait = TimeWait.builder()
.timeout(config.getLong(FlinkOptions.WRITE_COMMIT_ACK_TIMEOUT))
.action("instant initialize")
.throwsT(true)
.build();
while (instant == null || instant.equals(this.initInstant)) {
// wait condition:
@@ -202,7 +199,7 @@ public class BulkInsertWriteFunction<I>
// sleep for a while
timeWait.waitFor();
// refresh the inflight instant
instant = this.writeClient.getLastPendingInstant(this.actionType);
instant = StreamerUtil.getLastPendingInstant(this.metaClient);
}
return instant;
}

View File

@@ -20,9 +20,7 @@ package org.apache.hudi.sink.common;
import org.apache.hudi.client.HoodieFlinkWriteClient;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.util.CommitUtils;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.sink.StreamWriteOperatorCoordinator;
@@ -70,6 +68,11 @@ public abstract class AbstractStreamWriteFunction<I>
*/
protected int taskID;
/**
* Meta Client.
*/
protected transient HoodieTableMetaClient metaClient;
/**
* Write Client.
*/
@@ -85,11 +88,6 @@ public abstract class AbstractStreamWriteFunction<I>
*/
protected transient OperatorEventGateway eventGateway;
/**
* Commit action type.
*/
protected transient String actionType;
/**
* Flag saying whether the write task is waiting for the checkpoint success notification
* after it finished a checkpoint.
@@ -128,11 +126,8 @@ public abstract class AbstractStreamWriteFunction<I>
@Override
public void initializeState(FunctionInitializationContext context) throws Exception {
this.taskID = getRuntimeContext().getIndexOfThisSubtask();
this.metaClient = StreamerUtil.createMetaClient(this.config);
this.writeClient = StreamerUtil.createWriteClient(this.config, getRuntimeContext());
this.actionType = CommitUtils.getCommitActionType(
WriteOperationType.fromValue(config.getString(FlinkOptions.OPERATION)),
HoodieTableType.valueOf(config.getString(FlinkOptions.TABLE_TYPE)));
this.writeStatuses = new ArrayList<>();
this.writeMetadataState = context.getOperatorStateStore().getListState(
new ListStateDescriptor<>(
@@ -140,7 +135,7 @@ public abstract class AbstractStreamWriteFunction<I>
TypeInformation.of(WriteMetadataEvent.class)
));
this.currentInstant = this.writeClient.getLastPendingInstant(this.actionType);
this.currentInstant = lastPendingInstant();
if (context.isRestored()) {
restoreWriteMetadata();
} else {
@@ -162,12 +157,6 @@ public abstract class AbstractStreamWriteFunction<I>
// -------------------------------------------------------------------------
// Getter/Setter
// -------------------------------------------------------------------------
@VisibleForTesting
@SuppressWarnings("rawtypes")
public HoodieFlinkWriteClient getWriteClient() {
return writeClient;
}
@VisibleForTesting
public boolean isConfirming() {
return this.confirming;
@@ -182,7 +171,7 @@ public abstract class AbstractStreamWriteFunction<I>
// -------------------------------------------------------------------------
private void restoreWriteMetadata() throws Exception {
String lastInflight = this.writeClient.getLastPendingInstant(this.actionType);
String lastInflight = lastPendingInstant();
boolean eventSent = false;
for (WriteMetadataEvent event : this.writeMetadataState.get()) {
if (Objects.equals(lastInflight, event.getInstantTime())) {
@@ -224,6 +213,13 @@ public abstract class AbstractStreamWriteFunction<I>
this.confirming = false;
}
/**
* Returns the last pending instant time.
*/
protected String lastPendingInstant() {
return StreamerUtil.getLastPendingInstant(this.metaClient);
}
/**
* Prepares the instant time to write with for next checkpoint.
*
@@ -231,7 +227,7 @@ public abstract class AbstractStreamWriteFunction<I>
* @return The instant time
*/
protected String instantToWrite(boolean hasData) {
String instant = this.writeClient.getLastPendingInstant(this.actionType);
String instant = lastPendingInstant();
// if exactly-once semantics turns on,
// waits for the checkpoint notification until the checkpoint timeout threshold hits.
TimeWait timeWait = TimeWait.builder()
@@ -244,11 +240,18 @@ public abstract class AbstractStreamWriteFunction<I>
// 2. the inflight instant does not change and the checkpoint has buffering data
if (instant == null || (instant.equals(this.currentInstant) && hasData)) {
// sleep for a while
timeWait.waitFor();
// refresh the inflight instant
instant = this.writeClient.getLastPendingInstant(this.actionType);
boolean timeout = timeWait.waitFor();
if (timeout && instant != null) {
// if the timeout threshold hits but the last instant still not commit,
// and the task does not receive commit ask event(no data or aborted checkpoint),
// assumes the checkpoint was canceled silently and unblock the data flushing
confirming = false;
} else {
// refresh the inflight instant
instant = lastPendingInstant();
}
} else {
// the inflight instant changed, which means the last instant was committed
// the pending instant changed, that means the last instant was committed
// successfully.
confirming = false;
}

View File

@@ -69,7 +69,7 @@ public class HoodieFlinkCompactor {
// infer changelog mode
CompactionUtil.inferChangelogMode(conf, metaClient);
HoodieFlinkWriteClient writeClient = StreamerUtil.createWriteClient(conf, null);
HoodieFlinkWriteClient writeClient = StreamerUtil.createWriteClient(conf);
HoodieFlinkTable<?> table = writeClient.getHoodieTable();
// judge whether have operation
@@ -151,5 +151,6 @@ public class HoodieFlinkCompactor {
.setParallelism(1);
env.execute("flink_hudi_compaction");
writeClient.close();
}
}

View File

@@ -116,7 +116,7 @@ public class BucketAssignFunction<K, I, O extends HoodieRecord<?>>
@Override
public void open(Configuration parameters) throws Exception {
super.open(parameters);
HoodieWriteConfig writeConfig = StreamerUtil.getHoodieClientConfig(this.conf);
HoodieWriteConfig writeConfig = StreamerUtil.getHoodieClientConfig(this.conf, true);
HoodieFlinkEngineContext context = new HoodieFlinkEngineContext(
new SerializableConfiguration(StreamerUtil.getHadoopConf()),
new FlinkTaskContextSupplier(getRuntimeContext()));

View File

@@ -94,6 +94,11 @@ public class BucketAssigner implements AutoCloseable {
*/
private final Map<String, NewFileAssignState> newFileAssignStates;
/**
* Num of accumulated successful checkpoints, used for cleaning the new file assign state.
*/
private int accCkp = 0;
public BucketAssigner(
int taskID,
int maxParallelism,
@@ -117,7 +122,6 @@ public class BucketAssigner implements AutoCloseable {
*/
public void reset() {
bucketInfoMap.clear();
newFileAssignStates.clear();
}
public BucketInfo addUpdate(String partitionPath, String fileIdHint) {
@@ -136,16 +140,7 @@ public class BucketAssigner implements AutoCloseable {
// first try packing this into one of the smallFiles
if (smallFileAssign != null && smallFileAssign.assign()) {
final String key = StreamerUtil.generateBucketKey(partitionPath, smallFileAssign.getFileId());
// create a new bucket or reuse an existing bucket
BucketInfo bucketInfo;
if (bucketInfoMap.containsKey(key)) {
// Assigns an inserts to existing update bucket
bucketInfo = bucketInfoMap.get(key);
} else {
bucketInfo = addUpdate(partitionPath, smallFileAssign.getFileId());
}
return bucketInfo;
return new BucketInfo(BucketType.UPDATE, smallFileAssign.getFileId(), partitionPath);
}
// if we have anything more, create new insert buckets, like normal
@@ -154,7 +149,20 @@ public class BucketAssigner implements AutoCloseable {
if (newFileAssignState.canAssign()) {
newFileAssignState.assign();
final String key = StreamerUtil.generateBucketKey(partitionPath, newFileAssignState.fileId);
return bucketInfoMap.get(key);
if (bucketInfoMap.containsKey(key)) {
// the newFileAssignStates is cleaned asynchronously when received the checkpoint success notification,
// the records processed within the time range:
// (start checkpoint, checkpoint success(and instant committed))
// should still be assigned to the small buckets of last checkpoint instead of new one.
// the bucketInfoMap is cleaned when checkpoint starts.
// A promotion: when the HoodieRecord can record whether it is an UPDATE or INSERT,
// we can always return an UPDATE BucketInfo here, and there is no need to record the
// UPDATE bucket through calling #addUpdate.
return bucketInfoMap.get(key);
}
return new BucketInfo(BucketType.UPDATE, newFileAssignState.fileId, partitionPath);
}
}
BucketInfo bucketInfo = new BucketInfo(BucketType.INSERT, createFileIdOfThisTask(), partitionPath);
@@ -166,7 +174,7 @@ public class BucketAssigner implements AutoCloseable {
return bucketInfo;
}
private SmallFileAssign getSmallFileAssign(String partitionPath) {
private synchronized SmallFileAssign getSmallFileAssign(String partitionPath) {
if (smallFileAssignMap.containsKey(partitionPath)) {
return smallFileAssignMap.get(partitionPath);
}
@@ -186,7 +194,19 @@ public class BucketAssigner implements AutoCloseable {
/**
* Refresh the table state like TableFileSystemView and HoodieTimeline.
*/
public void reload(long checkpointId) {
public synchronized void reload(long checkpointId) {
this.accCkp += 1;
if (this.accCkp > 1) {
// do not clean the new file assignment state for the first checkpoint,
// this #reload calling is triggered by checkpoint success event, the coordinator
// also relies on the checkpoint success event to commit the inflight instant,
// and very possibly this component receives the notification before the coordinator,
// if we do the cleaning, the records processed within the time range:
// (start checkpoint, checkpoint success(and instant committed))
// would be assigned to a fresh new data bucket which is not the right behavior.
this.newFileAssignStates.clear();
this.accCkp = 0;
}
this.smallFileAssignMap.clear();
this.writeProfile.reload(checkpointId);
}

View File

@@ -25,6 +25,7 @@ import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieRecordLocation;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.view.AbstractTableFileSystemView;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.action.commit.SmallFile;
@@ -55,7 +56,7 @@ public class DeltaWriteProfile extends WriteProfile {
if (!commitTimeline.empty()) {
HoodieInstant latestCommitTime = commitTimeline.lastInstant().get();
// initialize the filesystem view based on the commit metadata
initFSViewIfNecessary(commitTimeline);
initFileSystemView();
// find smallest file in partition and append to it
List<FileSlice> allSmallFileSlices = new ArrayList<>();
// If we can index log files, we can add more inserts to log files for fileIds including those under
@@ -90,6 +91,10 @@ public class DeltaWriteProfile extends WriteProfile {
return smallFileLocations;
}
protected AbstractTableFileSystemView getFileSystemView() {
return (AbstractTableFileSystemView) this.table.getSliceView();
}
private long getTotalFileSize(FileSlice fileSlice) {
if (!fileSlice.getBaseFile().isPresent()) {
return convertLogFilesSizeToExpectedParquetSize(fileSlice.getLogFiles().collect(Collectors.toList()));

View File

@@ -25,7 +25,7 @@ import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieRecordLocation;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
import org.apache.hudi.common.table.view.AbstractTableFileSystemView;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.sink.partitioner.BucketAssigner;
import org.apache.hudi.table.HoodieFlinkTable;
@@ -36,7 +36,6 @@ import org.apache.hudi.util.StreamerUtil;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.core.fs.Path;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -45,7 +44,6 @@ import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.Stream;
@@ -98,7 +96,7 @@ public class WriteProfile {
/**
* The file system view cache for one checkpoint interval.
*/
protected HoodieTableFileSystemView fsView;
protected AbstractTableFileSystemView fsView;
/**
* Hadoop configuration.
@@ -194,7 +192,7 @@ public class WriteProfile {
if (!commitTimeline.empty()) { // if we have some commits
HoodieInstant latestCommitTime = commitTimeline.lastInstant().get();
// initialize the filesystem view based on the commit metadata
initFSViewIfNecessary(commitTimeline);
initFileSystemView();
List<HoodieBaseFile> allFiles = fsView
.getLatestBaseFilesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()).collect(Collectors.toList());
@@ -214,22 +212,16 @@ public class WriteProfile {
}
@VisibleForTesting
public void initFSViewIfNecessary(HoodieTimeline commitTimeline) {
public void initFileSystemView() {
if (fsView == null) {
cleanMetadataCache(commitTimeline.getInstants());
List<HoodieCommitMetadata> metadataList = commitTimeline.getInstants()
.map(instant ->
this.metadataCache.computeIfAbsent(
instant.getTimestamp(),
k -> WriteProfiles.getCommitMetadataSafely(config.getTableName(), basePath, instant, commitTimeline)
.orElse(null)))
.filter(Objects::nonNull)
.collect(Collectors.toList());
FileStatus[] commitFiles = WriteProfiles.getWritePathsOfInstants(basePath, hadoopConf, metadataList, table.getMetaClient().getTableType());
fsView = new HoodieTableFileSystemView(table.getMetaClient(), commitTimeline, commitFiles);
fsView = getFileSystemView();
}
}
protected AbstractTableFileSystemView getFileSystemView() {
return (AbstractTableFileSystemView) this.table.getBaseFileOnlyView();
}
/**
* Remove the overdue metadata from the cache
* whose instant does not belong to the given instants {@code instants}.
@@ -261,8 +253,10 @@ public class WriteProfile {
return;
}
this.table.getMetaClient().reloadActiveTimeline();
this.table.getHoodieView().sync();
recordProfile();
this.fsView = null;
cleanMetadataCache(this.table.getMetaClient().getCommitsTimeline().filterCompletedInstants().getInstants());
this.smallFilesMap.clear();
this.reloadedCheckpointId = checkpointId;
}

View File

@@ -20,6 +20,9 @@ package org.apache.hudi.sink.utils;
import org.apache.hudi.exception.HoodieException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Objects;
import java.util.concurrent.TimeUnit;
@@ -27,15 +30,19 @@ import java.util.concurrent.TimeUnit;
* Tool used for time waiting.
*/
public class TimeWait {
private final long timeout; // timeout in SECONDS
private final long interval; // interval in MILLISECONDS
private final String action; // action to report error message
private static final Logger LOG = LoggerFactory.getLogger(TimeWait.class);
private final long timeout; // timeout in SECONDS
private final long interval; // interval in MILLISECONDS
private final String action; // action to report error message
private final boolean throwsE; // whether to throw when timeout
private long waitingTime = 0L;
private TimeWait(long timeout, long interval, String action) {
private TimeWait(long timeout, long interval, String action, boolean throwsE) {
this.timeout = timeout;
this.interval = interval;
this.action = action;
this.throwsE = throwsE;
}
public static Builder builder() {
@@ -44,14 +51,23 @@ public class TimeWait {
/**
* Wait for an interval time.
*
* @return true if is timed out
*/
public void waitFor() {
public boolean waitFor() {
try {
if (waitingTime > timeout) {
throw new HoodieException("Timeout(" + waitingTime + "ms) while waiting for " + action);
final String msg = "Timeout(" + waitingTime + "ms) while waiting for " + action;
if (this.throwsE) {
throw new HoodieException(msg);
} else {
LOG.warn(msg);
return true;
}
}
TimeUnit.MILLISECONDS.sleep(interval);
waitingTime += interval;
return false;
} catch (InterruptedException e) {
throw new HoodieException("Error while waiting for " + action, e);
}
@@ -61,17 +77,18 @@ public class TimeWait {
* Builder.
*/
public static class Builder {
private long timeout;
private long interval;
private long timeout = 5 * 60 * 1000L; // default 5 minutes
private long interval = 1000;
private String action;
private boolean throwsT = false;
public Builder() {
this.timeout = 3600;
this.interval = 500;
private Builder() {
}
public Builder timeout(long timeout) {
this.timeout = timeout;
if (timeout > 0) {
this.timeout = timeout;
}
return this;
}
@@ -85,9 +102,14 @@ public class TimeWait {
return this;
}
public Builder throwsT(boolean throwsT) {
this.throwsT = throwsT;
return this;
}
public TimeWait build() {
Objects.requireNonNull(this.action);
return new TimeWait(this.timeout, this.interval, this.action);
return new TimeWait(this.timeout, this.interval, this.action, this.throwsT);
}
}
}

View File

@@ -63,9 +63,9 @@ public class HoodieTableSink implements DynamicTableSink, SupportsPartitioning,
return (DataStreamSinkProvider) dataStream -> {
// setup configuration
long ckpTimeout = dataStream.getExecutionEnvironment()
.getCheckpointConfig().getCheckpointTimeout();
conf.setLong(FlinkOptions.WRITE_COMMIT_ACK_TIMEOUT, ckpTimeout);
long ckpInterval = dataStream.getExecutionEnvironment()
.getCheckpointConfig().getCheckpointInterval();
conf.setLong(FlinkOptions.WRITE_COMMIT_ACK_TIMEOUT, ckpInterval * 5); // five checkpoints interval
RowType rowType = (RowType) schema.toSourceRowDataType().notNull().getLogicalType();

View File

@@ -32,6 +32,8 @@ import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.log.HoodieLogFormat;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ReflectionUtils;
import org.apache.hudi.common.util.ValidationUtils;
@@ -149,7 +151,21 @@ public class StreamerUtil {
return FlinkClientUtil.getHadoopConf();
}
/**
* Mainly used for tests.
*/
public static HoodieWriteConfig getHoodieClientConfig(Configuration conf) {
return getHoodieClientConfig(conf, false, false);
}
public static HoodieWriteConfig getHoodieClientConfig(Configuration conf, boolean loadFsViewStorageConfig) {
return getHoodieClientConfig(conf, false, loadFsViewStorageConfig);
}
public static HoodieWriteConfig getHoodieClientConfig(
Configuration conf,
boolean enableEmbeddedTimelineService,
boolean loadFsViewStorageConfig) {
HoodieWriteConfig.Builder builder =
HoodieWriteConfig.newBuilder()
.withEngineType(EngineType.FLINK)
@@ -194,13 +210,20 @@ public class StreamerUtil {
.withPayloadOrderingField(conf.getString(FlinkOptions.PRECOMBINE_FIELD))
.withPayloadEventTimeField(conf.getString(FlinkOptions.PRECOMBINE_FIELD))
.build())
.withEmbeddedTimelineServerEnabled(enableEmbeddedTimelineService)
.withEmbeddedTimelineServerReuseEnabled(true) // make write client embedded timeline service singleton
.withAutoCommit(false)
.withAllowOperationMetadataField(conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED))
.withProps(flinkConf2TypedProperties(conf));
.withProps(flinkConf2TypedProperties(conf))
.withSchema(getSourceSchema(conf).toString());
builder = builder.withSchema(getSourceSchema(conf).toString());
return builder.build();
HoodieWriteConfig writeConfig = builder.build();
if (loadFsViewStorageConfig) {
// do not use the builder to give a change for recovering the original fs view storage config
FileSystemViewStorageConfig viewStorageConfig = ViewStorageProperties.loadFromProperties(conf.getString(FlinkOptions.PATH));
writeConfig.setViewStorageConfig(viewStorageConfig);
}
return writeConfig;
}
/**
@@ -235,11 +258,11 @@ public class StreamerUtil {
* @param conf the configuration
* @throws IOException if errors happens when writing metadata
*/
public static void initTableIfNotExists(Configuration conf) throws IOException {
public static HoodieTableMetaClient initTableIfNotExists(Configuration conf) throws IOException {
final String basePath = conf.getString(FlinkOptions.PATH);
final org.apache.hadoop.conf.Configuration hadoopConf = StreamerUtil.getHadoopConf();
if (!tableExists(basePath, hadoopConf)) {
HoodieTableMetaClient.withPropertyBuilder()
HoodieTableMetaClient metaClient = HoodieTableMetaClient.withPropertyBuilder()
.setTableType(conf.getString(FlinkOptions.TABLE_TYPE))
.setTableName(conf.getString(FlinkOptions.TABLE_NAME))
.setRecordKeyFields(conf.getString(FlinkOptions.RECORD_KEY_FIELD, null))
@@ -250,9 +273,11 @@ public class StreamerUtil {
.setTimelineLayoutVersion(1)
.initTable(hadoopConf, basePath);
LOG.info("Table initialized under base path {}", basePath);
return metaClient;
} else {
LOG.info("Table [{}/{}] already exists, no need to initialize the table",
basePath, conf.getString(FlinkOptions.TABLE_NAME));
return StreamerUtil.createMetaClient(basePath, hadoopConf);
}
// Do not close the filesystem in order to use the CACHE,
// some filesystems release the handles in #close method.
@@ -305,7 +330,7 @@ public class StreamerUtil {
/**
* Creates the meta client for reader.
*
* <p>The streaming pipeline process is long running, so empty table path is allowed,
* <p>The streaming pipeline process is long-running, so empty table path is allowed,
* the reader would then check and refresh the meta client.
*
* @see org.apache.hudi.source.StreamReadMonitoringFunction
@@ -344,6 +369,8 @@ public class StreamerUtil {
/**
* Creates the Flink write client.
*
* <p>This expects to be used by client, the driver should start an embedded timeline server.
*/
public static HoodieFlinkWriteClient createWriteClient(Configuration conf, RuntimeContext runtimeContext) {
HoodieFlinkEngineContext context =
@@ -351,16 +378,22 @@ public class StreamerUtil {
new SerializableConfiguration(getHadoopConf()),
new FlinkTaskContextSupplier(runtimeContext));
return new HoodieFlinkWriteClient<>(context, getHoodieClientConfig(conf));
HoodieWriteConfig writeConfig = getHoodieClientConfig(conf, true);
return new HoodieFlinkWriteClient<>(context, writeConfig);
}
/**
* Creates the Flink write client.
*
* <p>This expects to be used by the driver, the client can then send requests for files view.
*
* <p>The task context supplier is a constant: the write token is always '0-1-0'.
*/
public static HoodieFlinkWriteClient createWriteClient(Configuration conf) {
return new HoodieFlinkWriteClient<>(HoodieFlinkEngineContext.DEFAULT, getHoodieClientConfig(conf));
public static HoodieFlinkWriteClient createWriteClient(Configuration conf) throws IOException {
HoodieWriteConfig writeConfig = getHoodieClientConfig(conf, true, false);
// create the filesystem view storage properties for client
ViewStorageProperties.createProperties(conf.getString(FlinkOptions.PATH), writeConfig.getViewStorageConfig());
return new HoodieFlinkWriteClient<>(HoodieFlinkEngineContext.DEFAULT, writeConfig);
}
/**
@@ -433,6 +466,27 @@ public class StreamerUtil {
return operationType == WriteOperationType.INSERT && !conf.getBoolean(FlinkOptions.INSERT_DEDUP);
}
public static String getLastPendingInstant(HoodieTableMetaClient metaClient) {
return getLastPendingInstant(metaClient, true);
}
public static String getLastPendingInstant(HoodieTableMetaClient metaClient, boolean reloadTimeline) {
if (reloadTimeline) {
metaClient.reloadActiveTimeline();
}
return metaClient.getCommitsTimeline().filterInflightsAndRequested()
.lastInstant()
.map(HoodieInstant::getTimestamp)
.orElse(null);
}
public static String getLastCompletedInstant(HoodieTableMetaClient metaClient) {
return metaClient.getCommitsTimeline().filterCompletedInstants()
.lastInstant()
.map(HoodieInstant::getTimestamp)
.orElse(null);
}
/**
* Returns whether there are successful commits on the timeline.
* @param metaClient The meta client

View File

@@ -0,0 +1,82 @@
/*
* 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.hudi.common.fs.FSUtils;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.Date;
import java.util.Properties;
import static org.apache.hudi.common.table.HoodieTableMetaClient.AUXILIARYFOLDER_NAME;
/**
* Helper class to read/write {@link FileSystemViewStorageConfig}.
*/
public class ViewStorageProperties {
private static final Logger LOG = LoggerFactory.getLogger(ViewStorageProperties.class);
private static final String FILE_NAME = "view_storage_conf.properties";
/**
* Initialize the {@link #FILE_NAME} meta file.
*/
public static void createProperties(
String basePath,
FileSystemViewStorageConfig config) throws IOException {
Path propertyPath = getPropertiesFilePath(basePath);
FileSystem fs = FSUtils.getFs(basePath, StreamerUtil.getHadoopConf());
try (FSDataOutputStream outputStream = fs.create(propertyPath)) {
config.getProps().store(outputStream,
"Filesystem view storage properties saved on " + new Date(System.currentTimeMillis()));
}
}
/**
* Read the {@link FileSystemViewStorageConfig} with given table base path.
*/
public static FileSystemViewStorageConfig loadFromProperties(String basePath) {
Path propertyPath = getPropertiesFilePath(basePath);
LOG.info("Loading filesystem view storage properties from " + propertyPath);
FileSystem fs = FSUtils.getFs(basePath, StreamerUtil.getHadoopConf());
Properties props = new Properties();
try {
try (FSDataInputStream inputStream = fs.open(propertyPath)) {
props.load(inputStream);
}
return FileSystemViewStorageConfig.newBuilder().fromProperties(props).build();
} catch (IOException e) {
throw new HoodieIOException("Could not load filesystem view storage properties from " + propertyPath, e);
}
}
private static Path getPropertiesFilePath(String basePath) {
String auxPath = basePath + Path.SEPARATOR + AUXILIARYFOLDER_NAME;
return new Path(auxPath, FILE_NAME);
}
}