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

[FLINK-36770][Connectors/AWS] make AWS sink writers use ResultHandler, move flink version to 1.20 #186

Open
wants to merge 1 commit into
base: main
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
4 changes: 2 additions & 2 deletions .github/workflows/nightly.yml
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ jobs:
if: github.repository_owner == 'apache'
strategy:
matrix:
flink: [1.19-SNAPSHOT, 1.20-SNAPSHOT]
flink: [1.20-SNAPSHOT]
java: [ '8, 11, 17']
uses: ./.github/workflows/common.yml
with:
Expand All @@ -38,7 +38,7 @@ jobs:
python_test:
strategy:
matrix:
flink: [1.19-SNAPSHOT, 1.20-SNAPSHOT]
flink: [1.20-SNAPSHOT]
uses: apache/flink-connector-shared-utils/.github/workflows/python_ci.yml@ci_utils
with:
flink_version: ${{ matrix.flink }}
Expand Down
4 changes: 2 additions & 2 deletions .github/workflows/push_pr.yml
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,7 @@ jobs:
uses: ./.github/workflows/common.yml
strategy:
matrix:
flink: [1.19.1, 1.20.0]
flink: [1.20.0]
java: [ '8, 11, 17']
with:
flink_version: ${{ matrix.flink }}
Expand All @@ -38,7 +38,7 @@ jobs:
python_test:
strategy:
matrix:
flink: [1.19.0, 1.20.0]
flink: [1.20.0]
uses: apache/flink-connector-shared-utils/.github/workflows/python_ci.yml@ci_utils
with:
flink_version: ${{ matrix.flink }}
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import org.apache.flink.connector.base.sink.writer.AsyncSinkWriter;
import org.apache.flink.connector.base.sink.writer.BufferedRequestState;
import org.apache.flink.connector.base.sink.writer.ElementConverter;
import org.apache.flink.connector.base.sink.writer.ResultHandler;
import org.apache.flink.connector.base.sink.writer.config.AsyncSinkWriterConfiguration;
import org.apache.flink.metrics.Counter;
import org.apache.flink.metrics.groups.SinkWriterMetricGroup;
Expand All @@ -45,7 +46,6 @@
import java.util.List;
import java.util.Properties;
import java.util.concurrent.CompletableFuture;
import java.util.function.Consumer;

import static org.apache.flink.connector.aws.util.AWSCredentialFatalExceptionClassifiers.getInvalidCredentialsExceptionClassifier;
import static org.apache.flink.connector.aws.util.AWSCredentialFatalExceptionClassifiers.getSdkClientMisconfiguredExceptionClassifier;
Expand Down Expand Up @@ -171,7 +171,7 @@ private static FirehoseAsyncClient createFirehoseClient(

@Override
protected void submitRequestEntries(
List<Record> requestEntries, Consumer<List<Record>> requestResult) {
List<Record> requestEntries, ResultHandler<Record> resultHandler) {

PutRecordBatchRequest batchRequest =
PutRecordBatchRequest.builder()
Expand All @@ -185,11 +185,11 @@ protected void submitRequestEntries(
future.whenComplete(
(response, err) -> {
if (err != null) {
handleFullyFailedRequest(err, requestEntries, requestResult);
handleFullyFailedRequest(err, requestEntries, resultHandler);
} else if (response.failedPutCount() > 0) {
handlePartiallyFailedRequest(response, requestEntries, requestResult);
handlePartiallyFailedRequest(response, requestEntries, resultHandler);
} else {
requestResult.accept(Collections.emptyList());
resultHandler.complete();
}
});
}
Expand All @@ -205,35 +205,37 @@ public void close() {
}

private void handleFullyFailedRequest(
Throwable err, List<Record> requestEntries, Consumer<List<Record>> requestResult) {
Throwable err, List<Record> requestEntries, ResultHandler<Record> resultHandler) {

numRecordsOutErrorsCounter.inc(requestEntries.size());
boolean isFatal = FIREHOSE_EXCEPTION_HANDLER.consumeIfFatal(err, getFatalExceptionCons());
boolean isFatal =
FIREHOSE_EXCEPTION_HANDLER.consumeIfFatal(
err, resultHandler::completeExceptionally);
if (isFatal) {
return;
}

if (failOnError) {
getFatalExceptionCons()
.accept(new KinesisFirehoseException.KinesisFirehoseFailFastException(err));
resultHandler.completeExceptionally(
new KinesisFirehoseException.KinesisFirehoseFailFastException(err));
return;
}

LOG.warn(
"KDF Sink failed to write and will retry {} entries to KDF",
requestEntries.size(),
err);
requestResult.accept(requestEntries);
resultHandler.retryForEntries(requestEntries);
}

private void handlePartiallyFailedRequest(
PutRecordBatchResponse response,
List<Record> requestEntries,
Consumer<List<Record>> requestResult) {
ResultHandler<Record> resultHandler) {
numRecordsOutErrorsCounter.inc(response.failedPutCount());
if (failOnError) {
getFatalExceptionCons()
.accept(new KinesisFirehoseException.KinesisFirehoseFailFastException());
resultHandler.completeExceptionally(
new KinesisFirehoseException.KinesisFirehoseFailFastException());
return;
}

Expand All @@ -248,6 +250,6 @@ private void handlePartiallyFailedRequest(
}
}

requestResult.accept(failedRequestEntries);
resultHandler.retryForEntries(failedRequestEntries);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import org.apache.flink.connector.base.sink.writer.AsyncSinkWriter;
import org.apache.flink.connector.base.sink.writer.BufferedRequestState;
import org.apache.flink.connector.base.sink.writer.ElementConverter;
import org.apache.flink.connector.base.sink.writer.ResultHandler;
import org.apache.flink.connector.base.sink.writer.config.AsyncSinkWriterConfiguration;
import org.apache.flink.connector.base.sink.writer.strategy.AIMDScalingStrategy;
import org.apache.flink.connector.base.sink.writer.strategy.CongestionControlRateLimitingStrategy;
Expand All @@ -48,7 +49,6 @@
import java.util.List;
import java.util.Properties;
import java.util.concurrent.CompletableFuture;
import java.util.function.Consumer;

import static org.apache.flink.connector.aws.util.AWSCredentialFatalExceptionClassifiers.getInvalidCredentialsExceptionClassifier;
import static org.apache.flink.connector.aws.util.AWSCredentialFatalExceptionClassifiers.getSdkClientMisconfiguredExceptionClassifier;
Expand Down Expand Up @@ -199,7 +199,7 @@ private static RateLimitingStrategy buildRateLimitingStrategy(
@Override
protected void submitRequestEntries(
List<PutRecordsRequestEntry> requestEntries,
Consumer<List<PutRecordsRequestEntry>> requestResult) {
ResultHandler<PutRecordsRequestEntry> resultHandler) {

PutRecordsRequest batchRequest =
PutRecordsRequest.builder()
Expand All @@ -213,11 +213,11 @@ protected void submitRequestEntries(
future.whenComplete(
(response, err) -> {
if (err != null) {
handleFullyFailedRequest(err, requestEntries, requestResult);
handleFullyFailedRequest(err, requestEntries, resultHandler);
} else if (response.failedRecordCount() > 0) {
handlePartiallyFailedRequest(response, requestEntries, requestResult);
handlePartiallyFailedRequest(response, requestEntries, resultHandler);
} else {
requestResult.accept(Collections.emptyList());
resultHandler.complete();
}
});
}
Expand All @@ -230,15 +230,15 @@ protected long getSizeInBytes(PutRecordsRequestEntry requestEntry) {
private void handleFullyFailedRequest(
Throwable err,
List<PutRecordsRequestEntry> requestEntries,
Consumer<List<PutRecordsRequestEntry>> requestResult) {
ResultHandler<PutRecordsRequestEntry> resultHandler) {
LOG.warn(
"KDS Sink failed to write and will retry {} entries to KDS",
requestEntries.size(),
err);
numRecordsOutErrorsCounter.inc(requestEntries.size());

if (isRetryable(err)) {
requestResult.accept(requestEntries);
if (isRetryable(err, resultHandler)) {
resultHandler.retryForEntries(requestEntries);
}
}

Expand All @@ -250,15 +250,15 @@ public void close() {
private void handlePartiallyFailedRequest(
PutRecordsResponse response,
List<PutRecordsRequestEntry> requestEntries,
Consumer<List<PutRecordsRequestEntry>> requestResult) {
ResultHandler<PutRecordsRequestEntry> resultHandler) {
LOG.warn(
"KDS Sink failed to write and will retry {} entries to KDS",
response.failedRecordCount());
numRecordsOutErrorsCounter.inc(response.failedRecordCount());

if (failOnError) {
getFatalExceptionCons()
.accept(new KinesisStreamsException.KinesisStreamsFailFastException());
resultHandler.completeExceptionally(
new KinesisStreamsException.KinesisStreamsFailFastException());
return;
}
List<PutRecordsRequestEntry> failedRequestEntries =
Expand All @@ -271,17 +271,19 @@ private void handlePartiallyFailedRequest(
}
}

requestResult.accept(failedRequestEntries);
resultHandler.retryForEntries(failedRequestEntries);
}

private boolean isRetryable(Throwable err) {
private boolean isRetryable(
Throwable err, ResultHandler<PutRecordsRequestEntry> resultHandler) {

if (!KINESIS_FATAL_EXCEPTION_CLASSIFIER.isFatal(err, getFatalExceptionCons())) {
if (!KINESIS_FATAL_EXCEPTION_CLASSIFIER.isFatal(
err, resultHandler::completeExceptionally)) {
return false;
}
if (failOnError) {
getFatalExceptionCons()
.accept(new KinesisStreamsException.KinesisStreamsFailFastException(err));
resultHandler.completeExceptionally(
new KinesisStreamsException.KinesisStreamsFailFastException(err));
return false;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import org.apache.flink.connector.base.sink.writer.AsyncSinkWriter;
import org.apache.flink.connector.base.sink.writer.BufferedRequestState;
import org.apache.flink.connector.base.sink.writer.ElementConverter;
import org.apache.flink.connector.base.sink.writer.ResultHandler;
import org.apache.flink.connector.dynamodb.sink.client.SdkClientProvider;
import org.apache.flink.connector.dynamodb.util.PrimaryKeyBuilder;
import org.apache.flink.metrics.Counter;
Expand All @@ -45,12 +46,10 @@

import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CompletableFuture;
import java.util.function.Consumer;

import static java.util.Collections.singletonMap;
import static org.apache.flink.connector.aws.util.AWSCredentialFatalExceptionClassifiers.getInvalidCredentialsExceptionClassifier;
Expand Down Expand Up @@ -160,7 +159,7 @@ public DynamoDbSinkWriter(
@Override
protected void submitRequestEntries(
List<DynamoDbWriteRequest> requestEntries,
Consumer<List<DynamoDbWriteRequest>> requestResultConsumer) {
ResultHandler<DynamoDbWriteRequest> resultHandler) {

List<WriteRequest> items = new ArrayList<>();

Expand Down Expand Up @@ -190,17 +189,17 @@ protected void submitRequestEntries(
future.whenComplete(
(response, err) -> {
if (err != null) {
handleFullyFailedRequest(err, requestEntries, requestResultConsumer);
handleFullyFailedRequest(err, requestEntries, resultHandler);
} else if (!CollectionUtil.isNullOrEmpty(response.unprocessedItems())) {
handlePartiallyUnprocessedRequest(response, requestResultConsumer);
handlePartiallyUnprocessedRequest(response, resultHandler);
} else {
requestResultConsumer.accept(Collections.emptyList());
resultHandler.complete();
}
});
}

private void handlePartiallyUnprocessedRequest(
BatchWriteItemResponse response, Consumer<List<DynamoDbWriteRequest>> requestResult) {
BatchWriteItemResponse response, ResultHandler<DynamoDbWriteRequest> resultHandler) {
List<DynamoDbWriteRequest> unprocessed = new ArrayList<>();

for (WriteRequest writeRequest : response.unprocessedItems().get(tableName)) {
Expand All @@ -211,32 +210,33 @@ private void handlePartiallyUnprocessedRequest(
numRecordsSendErrorsCounter.inc(unprocessed.size());
numRecordsSendPartialFailure.inc(unprocessed.size());

requestResult.accept(unprocessed);
resultHandler.retryForEntries(unprocessed);
}

private void handleFullyFailedRequest(
Throwable err,
List<DynamoDbWriteRequest> requestEntries,
Consumer<List<DynamoDbWriteRequest>> requestResult) {
ResultHandler<DynamoDbWriteRequest> resultHandler) {
LOG.warn(
"DynamoDB Sink failed to persist and will retry {} entries.",
requestEntries.size(),
err);
numRecordsSendErrorsCounter.inc(requestEntries.size());

if (isRetryable(err.getCause())) {
requestResult.accept(requestEntries);
if (isRetryable(err.getCause(), resultHandler)) {
resultHandler.retryForEntries(requestEntries);
}
}

private boolean isRetryable(Throwable err) {
private boolean isRetryable(Throwable err, ResultHandler<DynamoDbWriteRequest> resultHandler) {
// isFatal() is really isNotFatal()
if (!DYNAMODB_FATAL_EXCEPTION_CLASSIFIER.isFatal(err, getFatalExceptionCons())) {
if (!DYNAMODB_FATAL_EXCEPTION_CLASSIFIER.isFatal(
err, resultHandler::completeExceptionally)) {
return false;
}
if (failOnError) {
getFatalExceptionCons()
.accept(new DynamoDbSinkException.DynamoDbSinkFailFastException(err));
resultHandler.completeExceptionally(
new DynamoDbSinkException.DynamoDbSinkFailFastException(err));
return false;
}

Expand Down
Loading
Loading