Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -62,9 +62,7 @@ public CleanFunction(Configuration conf) {
public void open(Configuration parameters) throws Exception {
super.open(parameters);
if (conf.getBoolean(FlinkOptions.CLEAN_ASYNC_ENABLED)) {
// do not use the remote filesystem view because the async cleaning service
// local timeline is very probably to fall behind with the remote one.
this.writeClient = StreamerUtil.createWriteClient(conf, getRuntimeContext(), false);
this.writeClient = StreamerUtil.createWriteClient(conf, getRuntimeContext());
this.executor = NonThrownExecutor.builder(LOG).waitForTasksFinish(true).build();

if (conf.getString(FlinkOptions.OPERATION).equals(WriteOperationType.INSERT_OVERWRITE_TABLE.value())) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.sink.event.CommitAckEvent;
import org.apache.hudi.sink.event.WriteMetadataEvent;
import org.apache.hudi.sink.meta.CkpMetadata;
import org.apache.hudi.sink.utils.HiveSyncContext;
import org.apache.hudi.sink.utils.NonThrownExecutor;
import org.apache.hudi.util.CompactionUtil;
Expand Down Expand Up @@ -137,6 +138,16 @@ public class StreamWriteOperatorCoordinator
*/
private transient TableState tableState;

/**
* The checkpoint metadata.
*/
private CkpMetadata ckpMetadata;

/**
* Current checkpoint.
*/
private long checkpointId = -1;

/**
* Constructs a StreamingSinkOperatorCoordinator.
*
Expand Down Expand Up @@ -175,6 +186,8 @@ public void start() throws Exception {
if (tableState.syncMetadata) {
initMetadataSync();
}
this.ckpMetadata = CkpMetadata.getInstance(this.metaClient.getFs(), metaClient.getBasePath());
this.ckpMetadata.bootstrap(this.metaClient);
}

@Override
Expand All @@ -192,10 +205,14 @@ public void close() throws Exception {
writeClient.close();
}
this.eventBuffer = null;
if (this.ckpMetadata != null) {
this.ckpMetadata.close();
}
}

@Override
public void checkpointCoordinator(long checkpointId, CompletableFuture<byte[]> result) {
this.checkpointId = checkpointId;
executor.execute(
() -> {
try {
Expand Down Expand Up @@ -238,6 +255,15 @@ public void notifyCheckpointComplete(long checkpointId) {
);
}

@Override
public void notifyCheckpointAborted(long checkpointId) {
if (checkpointId == this.checkpointId) {
executor.execute(() -> {
this.ckpMetadata.abortInstant(this.instant);
}, "abort instant %s", this.instant);
}
}

@Override
public void resetToCheckpoint(long checkpointID, byte[] checkpointData) {
// no operation
Expand Down Expand Up @@ -340,6 +366,7 @@ private void startInstant() {
// because the instant request from write task is asynchronous.
this.instant = this.writeClient.startCommit();
this.metaClient.getActiveTimeline().transitionRequestedToInflight(tableState.commitAction, this.instant);
this.ckpMetadata.startInstant(this.instant);
LOG.info("Create instant [{}] for table [{}] with type [{}]", this.instant,
this.conf.getString(FlinkOptions.TABLE_NAME), conf.getString(FlinkOptions.TABLE_TYPE));
}
Expand Down Expand Up @@ -488,6 +515,7 @@ private void doCommit(String instant, List<WriteStatus> writeResults) {
tableState.commitAction, partitionToReplacedFileIds);
if (success) {
reset();
this.ckpMetadata.commitInstant(instant);
LOG.info("Commit instant [{}] success!", instant);
} else {
throw new HoodieException(String.format("Commit instant [%s] failed!", instant));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@
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.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;
Expand All @@ -37,6 +36,7 @@
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.sink.bootstrap.aggregate.BootstrapAggFunction;
import org.apache.hudi.sink.meta.CkpMetadata;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.format.FormatUtils;
import org.apache.hudi.util.FlinkTables;
Expand Down Expand Up @@ -83,6 +83,8 @@ public class BootstrapOperator<I, O extends HoodieRecord<?>>

protected HoodieTable<?, ?, ?, ?> hoodieTable;

private CkpMetadata ckpMetadata;

protected final Configuration conf;

protected transient org.apache.hadoop.conf.Configuration hadoopConf;
Expand All @@ -101,8 +103,7 @@ public BootstrapOperator(Configuration conf) {

@Override
public void snapshotState(StateSnapshotContext context) throws Exception {
HoodieTableMetaClient metaClient = StreamerUtil.createMetaClient(this.conf);
lastInstantTime = StreamerUtil.getLastPendingInstant(metaClient);
lastInstantTime = this.ckpMetadata.lastPendingInstant();
instantState.update(Collections.singletonList(lastInstantTime));
}

Expand All @@ -124,6 +125,7 @@ public void initializeState(StateInitializationContext context) throws Exception
this.hadoopConf = StreamerUtil.getHadoopConf();
this.writeConfig = StreamerUtil.getHoodieClientConfig(this.conf, true);
this.hoodieTable = FlinkTables.createTable(writeConfig, hadoopConf, getRuntimeContext());
this.ckpMetadata = CkpMetadata.getInstance(hoodieTable.getMetaClient().getFs(), this.writeConfig.getBasePath());
this.aggregateManager = getRuntimeContext().getGlobalAggregateManager();

preLoadIndexRecords();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,11 +21,11 @@
import org.apache.hudi.client.HoodieFlinkWriteClient;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.model.WriteOperationType;
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;
import org.apache.hudi.sink.event.WriteMetadataEvent;
import org.apache.hudi.sink.meta.CkpMetadata;
import org.apache.hudi.sink.utils.TimeWait;
import org.apache.hudi.util.StreamerUtil;

Expand Down Expand Up @@ -79,11 +79,6 @@ public class BulkInsertWriteFunction<I>
*/
private int taskID;

/**
* Meta Client.
*/
private transient HoodieTableMetaClient metaClient;

/**
* Write Client.
*/
Expand All @@ -99,6 +94,11 @@ public class BulkInsertWriteFunction<I>
*/
private transient OperatorEventGateway eventGateway;

/**
* Checkpoint metadata.
*/
private CkpMetadata ckpMetadata;

/**
* Constructs a StreamingSinkFunction.
*
Expand All @@ -112,9 +112,9 @@ public BulkInsertWriteFunction(Configuration config, RowType rowType) {
@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.initInstant = StreamerUtil.getLastPendingInstant(this.metaClient, false);
this.ckpMetadata = CkpMetadata.getInstance(config.getString(FlinkOptions.PATH));
this.initInstant = lastPendingInstant();
sendBootstrapEvent();
initWriterHelper();
}
Expand Down Expand Up @@ -187,7 +187,7 @@ private void sendBootstrapEvent() {
* Returns the last pending instant time.
*/
protected String lastPendingInstant() {
return StreamerUtil.getLastPendingInstant(this.metaClient);
return this.ckpMetadata.lastPendingInstant();
}

private String instantToWrite() {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import org.apache.hudi.sink.StreamWriteOperatorCoordinator;
import org.apache.hudi.sink.event.CommitAckEvent;
import org.apache.hudi.sink.event.WriteMetadataEvent;
import org.apache.hudi.sink.meta.CkpMetadata;
import org.apache.hudi.sink.utils.TimeWait;
import org.apache.hudi.util.StreamerUtil;

Expand Down Expand Up @@ -114,6 +115,11 @@ public abstract class AbstractStreamWriteFunction<I>
*/
protected List<WriteStatus> writeStatuses;

/**
* The checkpoint metadata.
*/
private transient CkpMetadata ckpMetadata;

/**
* Constructs a StreamWriteFunctionBase.
*
Expand All @@ -135,6 +141,7 @@ public void initializeState(FunctionInitializationContext context) throws Except
TypeInformation.of(WriteMetadataEvent.class)
));

this.ckpMetadata = CkpMetadata.getInstance(this.metaClient.getFs(), this.metaClient.getBasePath());
this.currentInstant = lastPendingInstant();
if (context.isRestored()) {
restoreWriteMetadata();
Expand Down Expand Up @@ -217,7 +224,7 @@ public void handleOperatorEvent(OperatorEvent event) {
* Returns the last pending instant time.
*/
protected String lastPendingInstant() {
return StreamerUtil.getLastPendingInstant(this.metaClient);
return this.ckpMetadata.lastPendingInstant();
}

/**
Expand All @@ -238,7 +245,7 @@ protected String instantToWrite(boolean hasData) {
// wait condition:
// 1. there is no inflight instant
// 2. the inflight instant does not change and the checkpoint has buffering data
if (instant == null || (instant.equals(this.currentInstant) && hasData)) {
if (instant == null || (instant.equals(this.currentInstant) && hasData && !this.ckpMetadata.isAborted(instant))) {
// sleep for a while
timeWait.waitFor();
// refresh the inflight instant
Expand Down
113 changes: 113 additions & 0 deletions hudi-flink/src/main/java/org/apache/hudi/sink/meta/CkpMessage.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,113 @@
/*
* 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.sink.meta;

import org.apache.hudi.common.util.ValidationUtils;

import org.apache.hadoop.fs.FileStatus;
import org.jetbrains.annotations.NotNull;

import java.io.Serializable;
import java.util.Arrays;
import java.util.Comparator;
import java.util.List;
import java.util.stream.Collectors;

/**
* A checkpoint message.
*/
public class CkpMessage implements Serializable, Comparable<CkpMessage> {
private static final long serialVersionUID = 1L;

public static final Comparator<CkpMessage> COMPARATOR = Comparator.comparing(CkpMessage::getInstant)
.thenComparing(CkpMessage::getState);

private final String instant; // the instant time
private final State state; // the checkpoint state

public CkpMessage(String instant, String state) {
this.instant = instant;
this.state = State.valueOf(state);
}

public CkpMessage(FileStatus fileStatus) {
String fileName = fileStatus.getPath().getName();
String[] nameAndExt = fileName.split("\\.");
ValidationUtils.checkState(nameAndExt.length == 2);
String name = nameAndExt[0];
String ext = nameAndExt[1];

this.instant = name;
this.state = State.valueOf(ext);
}

public String getInstant() {
return instant;
}

public State getState() {
return state;
}

public boolean isAborted() {
return State.ABORTED == this.state;
}

public boolean isComplete() {
return State.COMPLETED == this.state;
}

public boolean isInflight() {
return State.INFLIGHT == this.state;
}

public static String getFileName(String instant, State state) {
return instant + "." + state.name();
}

public static List<String> getAllFileNames(String instant) {
return Arrays.stream(State.values())
.map(state -> getFileName(instant, state))
.collect(Collectors.toList());
}

@Override
public int compareTo(@NotNull CkpMessage o) {
return COMPARATOR.compare(this, o);
}

/**
* Instant State.
*/
public enum State {
// Inflight instant
INFLIGHT,
// Aborted instant
// An instant can be aborted then be reused again, so it has lower priority
// than COMPLETED
ABORTED,
// Committed instant
COMPLETED
}

@Override
public String toString() {
return "Ckp{" + "instant='" + instant + '\'' + ", state='" + state + '\'' + '}';
}
}
Loading