Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 98F9D200498 for ; Tue, 29 Aug 2017 14:47:01 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 974201622F4; Tue, 29 Aug 2017 12:47:01 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id ACFBF161D94 for ; Tue, 29 Aug 2017 14:47:00 +0200 (CEST) Received: (qmail 31366 invoked by uid 500); 29 Aug 2017 12:46:59 -0000 Mailing-List: contact commits-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list commits@flink.apache.org Received: (qmail 31331 invoked by uid 99); 29 Aug 2017 12:46:59 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 29 Aug 2017 12:46:59 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 8EB36F32D5; Tue, 29 Aug 2017 12:46:58 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: aljoscha@apache.org To: commits@flink.apache.org Date: Tue, 29 Aug 2017 12:46:58 -0000 Message-Id: In-Reply-To: <93a0781368204e0a8a31cde11f0df854@git.apache.org> References: <93a0781368204e0a8a31cde11f0df854@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [2/4] flink git commit: [FLINK-7497][streaming] Introduce user context in TwoPhaseCommitSinkFunction archived-at: Tue, 29 Aug 2017 12:47:01 -0000 [FLINK-7497][streaming] Introduce user context in TwoPhaseCommitSinkFunction Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/959d54fc Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/959d54fc Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/959d54fc Branch: refs/heads/master Commit: 959d54fc828691759f15f2e83c0c123e9da6e782 Parents: ac72360 Author: Piotr Nowojski Authored: Mon Aug 21 16:53:07 2017 +0200 Committer: Aljoscha Krettek Committed: Tue Aug 29 14:45:34 2017 +0200 ---------------------------------------------------------------------- .../sink/TwoPhaseCommitSinkFunction.java | 57 ++++++++++++++++---- .../sink/TwoPhaseCommitSinkFunctionTest.java | 4 +- 2 files changed, 48 insertions(+), 13 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/flink/blob/959d54fc/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java ---------------------------------------------------------------------- diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java index b73272d..77f74fe 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java @@ -37,6 +37,7 @@ import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import static java.util.Objects.requireNonNull; import static org.apache.flink.util.Preconditions.checkState; @@ -49,22 +50,25 @@ import static org.apache.flink.util.Preconditions.checkState; * * @param Input type for {@link SinkFunction}. * @param Transaction to store all of the information required to handle a transaction. + * @param Context that will be shared across all invocations for the given {@link TwoPhaseCommitSinkFunction} + * instance. Context is created once */ @PublicEvolving -public abstract class TwoPhaseCommitSinkFunction +public abstract class TwoPhaseCommitSinkFunction extends RichSinkFunction implements CheckpointedFunction, CheckpointListener { private static final Logger LOG = LoggerFactory.getLogger(TwoPhaseCommitSinkFunction.class); - protected final ListStateDescriptor> stateDescriptor; + protected final ListStateDescriptor> stateDescriptor; protected final LinkedHashMap pendingCommitTransactions = new LinkedHashMap<>(); @Nullable protected TXN currentTransaction; + protected Optional userContext; - protected ListState> state; + protected ListState> state; /** * Use default {@link ListStateDescriptor} for internal state serialization. Helpful utilities for using this @@ -77,8 +81,8 @@ public abstract class TwoPhaseCommitSinkFunction * * @param stateTypeInformation {@link TypeInformation} for POJO holding state of opened transactions. */ - public TwoPhaseCommitSinkFunction(TypeInformation> stateTypeInformation) { - this(new ListStateDescriptor>("state", stateTypeInformation)); + public TwoPhaseCommitSinkFunction(TypeInformation> stateTypeInformation) { + this(new ListStateDescriptor>("state", stateTypeInformation)); } /** @@ -86,10 +90,18 @@ public abstract class TwoPhaseCommitSinkFunction * * @param stateDescriptor descriptor for transactions POJO. */ - public TwoPhaseCommitSinkFunction(ListStateDescriptor> stateDescriptor) { + public TwoPhaseCommitSinkFunction(ListStateDescriptor> stateDescriptor) { this.stateDescriptor = requireNonNull(stateDescriptor, "stateDescriptor is null"); } + protected Optional initializeUserContext() { + return Optional.empty(); + } + + protected Optional getUserContext() { + return userContext; + } + // ------ methods that should be implemented in child class to support two phase commit algorithm ------ /** @@ -142,6 +154,9 @@ public abstract class TwoPhaseCommitSinkFunction abort(transaction); } + protected void finishRecoveringContext() { + } + // ------ entry points for above methods implementing {@CheckPointedFunction} and {@CheckpointListener} ------ @Override @@ -226,7 +241,8 @@ public abstract class TwoPhaseCommitSinkFunction state.clear(); state.add(new State<>( this.currentTransaction, - new ArrayList<>(pendingCommitTransactions.values()))); + new ArrayList<>(pendingCommitTransactions.values()), + userContext)); } @Override @@ -250,7 +266,8 @@ public abstract class TwoPhaseCommitSinkFunction if (context.isRestored()) { LOG.info("{} - restoring state", name()); - for (State operatorState : state.get()) { + for (State operatorState : state.get()) { + userContext = operatorState.getContext(); List recoveredTransactions = operatorState.getPendingCommitTransactions(); for (TXN recoveredTransaction : recoveredTransactions) { // If this fails, there is actually a data loss @@ -260,9 +277,17 @@ public abstract class TwoPhaseCommitSinkFunction recoverAndAbort(operatorState.getPendingTransaction()); LOG.info("{} aborted recovered transaction {}", name(), operatorState.getPendingTransaction()); + + if (userContext.isPresent()) { + finishRecoveringContext(); + } } - } else { + } + // if in restore we didn't get any userContext or we are initializing from scratch + if (userContext == null) { LOG.info("{} - no state to restore {}", name()); + + userContext = initializeUserContext(); } this.pendingCommitTransactions.clear(); @@ -291,14 +316,16 @@ public abstract class TwoPhaseCommitSinkFunction /** * State POJO class coupling pendingTransaction, context and pendingCommitTransactions. */ - public static class State { + public static class State { protected TXN pendingTransaction; protected List pendingCommitTransactions = new ArrayList<>(); + protected Optional context; public State() { } - public State(TXN pendingTransaction, List pendingCommitTransactions) { + public State(TXN pendingTransaction, List pendingCommitTransactions, Optional context) { + this.context = requireNonNull(context, "context is null"); this.pendingTransaction = requireNonNull(pendingTransaction, "pendingTransaction is null"); this.pendingCommitTransactions = requireNonNull(pendingCommitTransactions, "pendingCommitTransactions is null"); } @@ -318,5 +345,13 @@ public abstract class TwoPhaseCommitSinkFunction public void setPendingCommitTransactions(List pendingCommitTransactions) { this.pendingCommitTransactions = pendingCommitTransactions; } + + public Optional getContext() { + return context; + } + + public void setContext(Optional context) { + this.context = context; + } } } http://git-wip-us.apache.org/repos/asf/flink/blob/959d54fc/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunctionTest.java ---------------------------------------------------------------------- diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunctionTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunctionTest.java index f2fcb96..b9097d7 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunctionTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunctionTest.java @@ -120,12 +120,12 @@ public class TwoPhaseCommitSinkFunctionTest { assertEquals(expectedValues, actualValues); } - private static class FileBasedSinkFunction extends TwoPhaseCommitSinkFunction { + private static class FileBasedSinkFunction extends TwoPhaseCommitSinkFunction { private final File tmpDirectory; private final File targetDirectory; public FileBasedSinkFunction(File tmpDirectory, File targetDirectory) { - super(TypeInformation.of(new TypeHint>() {})); + super(TypeInformation.of(new TypeHint>() {})); if (!tmpDirectory.isDirectory() || !targetDirectory.isDirectory()) { throw new IllegalArgumentException();