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 7C0CF200CF3 for ; Tue, 29 Aug 2017 14:47:01 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 7A46A161889; 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 CC2D31622F4 for ; Tue, 29 Aug 2017 14:47:00 +0200 (CEST) Received: (qmail 31558 invoked by uid 500); 29 Aug 2017 12:47:00 -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 31347 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 91B12F32EA; 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:59 -0000 Message-Id: <87c984a8327342e391d3d6ee039c2b2c@git.apache.org> In-Reply-To: <93a0781368204e0a8a31cde11f0df854@git.apache.org> References: <93a0781368204e0a8a31cde11f0df854@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [3/4] flink git commit: [hotfix][streaming] Allow to override methods from TwoPhaseCommitSinkFunction archived-at: Tue, 29 Aug 2017 12:47:01 -0000 [hotfix][streaming] Allow to override methods from TwoPhaseCommitSinkFunction This allow for some custom user logic during handling checkpoints. Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/9d9cdcba Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/9d9cdcba Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/9d9cdcba Branch: refs/heads/master Commit: 9d9cdcbad6ccf353a1252866f6a56ac505bfaa95 Parents: 959d54f Author: Piotr Nowojski Authored: Mon Aug 14 15:45:45 2017 +0200 Committer: Aljoscha Krettek Committed: Tue Aug 29 14:45:34 2017 +0200 ---------------------------------------------------------------------- .../api/functions/sink/TwoPhaseCommitSinkFunction.java | 4 ++-- .../api/functions/sink/TwoPhaseCommitSinkFunctionTest.java | 5 ++++- 2 files changed, 6 insertions(+), 3 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/flink/blob/9d9cdcba/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 77f74fe..18f74b6 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 @@ -222,7 +222,7 @@ public abstract class TwoPhaseCommitSinkFunction } @Override - public final void snapshotState(FunctionSnapshotContext context) throws Exception { + public void snapshotState(FunctionSnapshotContext context) throws Exception { // this is like the pre-commit of a 2-phase-commit transaction // we are ready to commit and remember the transaction @@ -246,7 +246,7 @@ public abstract class TwoPhaseCommitSinkFunction } @Override - public final void initializeState(FunctionInitializationContext context) throws Exception { + public void initializeState(FunctionInitializationContext context) throws Exception { // when we are restoring state with pendingCommitTransactions, we don't really know whether the // transactions were already committed, or whether there was a failure between // completing the checkpoint on the master, and notifying the writer here. http://git-wip-us.apache.org/repos/asf/flink/blob/9d9cdcba/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 b9097d7..4715c39 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 @@ -155,7 +155,10 @@ public class TwoPhaseCommitSinkFunctionTest { @Override protected void commit(FileTransaction transaction) { try { - Files.move(transaction.tmpFile.toPath(), new File(targetDirectory, transaction.tmpFile.getName()).toPath(), ATOMIC_MOVE); + Files.move( + transaction.tmpFile.toPath(), + new File(targetDirectory, transaction.tmpFile.getName()).toPath(), + ATOMIC_MOVE); } catch (IOException e) { throw new IllegalStateException(e); }