Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[hudi-9041]send act commit event when reuse current instant #12849

Open
wants to merge 4 commits into
base: master
Choose a base branch
from
Open
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 @@ -449,6 +449,9 @@ private void handleBootstrapEvent(WriteMetadataEvent event) {
LOG.warn("Reuse current pending Instant {} with {} operationType, "
+ "ignoring empty bootstrap event.", this.instant, WriteOperationType.INSERT.value());
reset();

// send commit act event to unblock write tasks
sendCommitAckEvents(-1L);
return;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -126,6 +126,30 @@ public void testSubtaskFails() throws Exception {
.end();
}

@Test
public void testAppendInsertAfterFailoverWithEmptyCheckpoint() throws Exception {
// open the function and ingest data
conf.setLong(FlinkOptions.WRITE_COMMIT_ACK_TIMEOUT, 10_000L);
conf.setString(FlinkOptions.OPERATION, "INSERT");
preparePipeline()
.assertEmptyDataFiles()
// make an empty snapshot
.checkpoint(1)
.assertEmptyEvent()
// trigger a partial failure
.subTaskFails(0, 1)
.assertNextEvent()
// make sure coordinator send an ack event to unblock the writers.
.assertNextSubTaskEvent()
// write a set of data and check the result.
.consume(TestData.DATA_SET_INSERT)
.checkpoint(2)
.assertNextEvent()
.checkpointComplete(2)
.checkWrittenData(EXPECTED1)
.end();
}

// Only when Job level fails with INSERT operationType can we roll back the unfinished instant.
// Task level failed retry, we should reuse the unfinished Instant with INSERT operationType
@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import org.apache.hudi.sink.StreamWriteFunction;
import org.apache.hudi.sink.StreamWriteOperatorCoordinator;
import org.apache.hudi.sink.bucket.BucketStreamWriteFunction;
import org.apache.hudi.sink.common.AbstractWriteFunction;
import org.apache.hudi.sink.event.WriteMetadataEvent;
import org.apache.hudi.sink.transform.RowDataToHoodieFunction;
import org.apache.hudi.util.AvroSchemaConverter;
Expand Down Expand Up @@ -193,6 +194,11 @@ public StreamWriteOperatorCoordinator getCoordinator() {
return coordinator;
}

@Override
public AbstractWriteFunction getWriteFunction() {
return this.writeFunction;
}

public MockOperatorCoordinatorContext getCoordinatorContext() {
return coordinatorContext;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@
import org.apache.hudi.sink.bulk.RowDataKeyGen;
import org.apache.hudi.sink.bulk.sort.SortOperator;
import org.apache.hudi.sink.bulk.sort.SortOperatorGen;
import org.apache.hudi.sink.common.AbstractWriteFunction;
import org.apache.hudi.sink.event.WriteMetadataEvent;
import org.apache.hudi.util.AvroSchemaConverter;
import org.apache.hudi.util.StreamerUtil;
Expand Down Expand Up @@ -176,6 +177,11 @@ public StreamWriteOperatorCoordinator getCoordinator() {
return coordinator;
}

@Override
public AbstractWriteFunction getWriteFunction() {
return this.writeFunction;
}

public MockOperatorCoordinatorContext getCoordinatorContext() {
return coordinatorContext;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import org.apache.hudi.sink.StreamWriteOperatorCoordinator;
import org.apache.hudi.sink.append.AppendWriteFunction;
import org.apache.hudi.sink.bulk.BulkInsertWriterHelper;
import org.apache.hudi.sink.common.AbstractWriteFunction;
import org.apache.hudi.sink.event.WriteMetadataEvent;
import org.apache.hudi.util.AvroSchemaConverter;
import org.apache.hudi.util.StreamerUtil;
Expand Down Expand Up @@ -58,6 +59,7 @@ public class InsertFunctionWrapper<I> implements TestFunctionWrapper<I> {

private final MockStreamingRuntimeContext runtimeContext;
private final MockOperatorEventGateway gateway;
private final MockSubtaskGateway subtaskGateway;
private final MockOperatorCoordinatorContext coordinatorContext;
private StreamWriteOperatorCoordinator coordinator;
private final MockStateInitializationContext stateInitializationContext;
Expand All @@ -79,6 +81,7 @@ public InsertFunctionWrapper(String tablePath, Configuration conf) throws Except
.build();
this.runtimeContext = new MockStreamingRuntimeContext(false, 1, 0, environment);
this.gateway = new MockOperatorEventGateway();
this.subtaskGateway = new MockSubtaskGateway();
this.conf = conf;
this.rowType = (RowType) AvroSchemaConverter.convertToDataType(StreamerUtil.getSourceSchema(conf)).getLogicalType();
// one function
Expand Down Expand Up @@ -121,6 +124,11 @@ public OperatorEvent getNextEvent() {
return this.gateway.getNextEvent();
}

@Override
public OperatorEvent getNextSubTaskEvent() {
return this.subtaskGateway.getNextEvent();
}

public void checkpointFunction(long checkpointId) throws Exception {
// checkpoint the coordinator first
this.coordinator.checkpointCoordinator(checkpointId, new CompletableFuture<>());
Expand Down Expand Up @@ -174,6 +182,11 @@ public StreamWriteOperatorCoordinator getCoordinator() {
return coordinator;
}

@Override
public AbstractWriteFunction getWriteFunction() {
return this.writeFunction;
}

@Override
public void close() throws Exception {
this.coordinator.close();
Expand All @@ -196,5 +209,7 @@ private void setupWriteFunction() throws Exception {
writeFunction.setOperatorEventGateway(gateway);
writeFunction.initializeState(this.stateInitializationContext);
writeFunction.open(conf);
// set up subtask gateway
coordinator.subtaskReady(0, subtaskGateway);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,55 @@
/*
* 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.utils;

import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
import org.apache.flink.runtime.messages.Acknowledge;
import org.apache.flink.runtime.operators.coordination.OperatorCoordinator;
import org.apache.flink.runtime.operators.coordination.OperatorEvent;

import java.util.LinkedList;
import java.util.concurrent.CompletableFuture;

/**
* A mock {@link OperatorCoordinator.SubtaskGateway} for unit tests.
*/
public class MockSubtaskGateway implements OperatorCoordinator.SubtaskGateway {

private final LinkedList<OperatorEvent> events = new LinkedList<>();

@Override
public CompletableFuture<Acknowledge> sendEvent(OperatorEvent operatorEvent) {
events.add(operatorEvent);
return CompletableFuture.completedFuture(Acknowledge.get());
}

@Override
public ExecutionAttemptID getExecution() {
return null;
}

@Override
public int getSubtask() {
return 0;
}

public OperatorEvent getNextEvent() {
return this.events.isEmpty() ? null : this.events.removeFirst();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import org.apache.hudi.sink.StreamWriteFunction;
import org.apache.hudi.sink.StreamWriteOperatorCoordinator;
import org.apache.hudi.sink.bootstrap.BootstrapOperator;
import org.apache.hudi.sink.common.AbstractWriteFunction;
import org.apache.hudi.sink.event.WriteMetadataEvent;
import org.apache.hudi.sink.partitioner.BucketAssignFunction;
import org.apache.hudi.sink.transform.RowDataToHoodieFunction;
Expand Down Expand Up @@ -273,6 +274,11 @@ public StreamWriteOperatorCoordinator getCoordinator() {
return coordinator;
}

@Override
public AbstractWriteFunction getWriteFunction() {
return this.writeFunction;
}

public MockOperatorCoordinatorContext getCoordinatorContext() {
return coordinatorContext;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.sink.StreamWriteOperatorCoordinator;
import org.apache.hudi.sink.common.AbstractWriteFunction;
import org.apache.hudi.sink.event.WriteMetadataEvent;

import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext;
Expand Down Expand Up @@ -49,10 +50,17 @@ public interface TestFunctionWrapper<I> {
WriteMetadataEvent[] getEventBuffer();

/**
* Returns the next event.
* Returns the next event sent to Coordinator.
*/
OperatorEvent getNextEvent();

/**
* Returns the next event sent to subtask.
*/
default OperatorEvent getNextSubTaskEvent() {
throw new UnsupportedOperationException();
}

/**
* Snapshot all the functions in the wrapper.
*/
Expand Down Expand Up @@ -103,6 +111,11 @@ default void restartCoordinator() throws Exception {
*/
StreamWriteOperatorCoordinator getCoordinator();

/**
* Returns the write function.
*/
AbstractWriteFunction getWriteFunction();

/**
* Returns the data buffer of the write task.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.configuration.OptionsResolver;
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.meta.CkpMetadataFactory;
Expand Down Expand Up @@ -206,6 +207,18 @@ public TestHarness emptyEventBuffer() {
return this;
}

/**
* Assert the next event exists and handle over it to the coordinator.
*/
public TestHarness assertNextSubTaskEvent() {
final OperatorEvent nextEvent = this.pipeline.getNextSubTaskEvent();
if (nextEvent != null) {
MatcherAssert.assertThat("The Coordinator expect to send an event", nextEvent, instanceOf(CommitAckEvent.class));
this.pipeline.getWriteFunction().handleOperatorEvent(nextEvent);
}
return this;
}

/**
* Assert the next event exists and handle over it to the coordinator.
*/
Expand Down