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 2473C200BE2 for ; Thu, 15 Dec 2016 11:22:43 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 22FF6160B10; Thu, 15 Dec 2016 10:22:43 +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 4250B160B0B for ; Thu, 15 Dec 2016 11:22:42 +0100 (CET) Received: (qmail 87107 invoked by uid 500); 15 Dec 2016 10:22:41 -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 87098 invoked by uid 99); 15 Dec 2016 10:22:41 -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; Thu, 15 Dec 2016 10:22:41 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 2F4B9E3908; Thu, 15 Dec 2016 10:22:41 +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 Message-Id: <3b4c01e8ece44053ba9e7b9ab1121c32@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: flink git commit: [FLINK-5335] Allow ListCheckpointed user functions to return null Date: Thu, 15 Dec 2016 10:22:41 +0000 (UTC) archived-at: Thu, 15 Dec 2016 10:22:43 -0000 Repository: flink Updated Branches: refs/heads/master 09e081730 -> 5dab9345c [FLINK-5335] Allow ListCheckpointed user functions to return null Project: http://git-wip-us.apache.org/repos/asf/flink/repo Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/5dab9345 Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/5dab9345 Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/5dab9345 Branch: refs/heads/master Commit: 5dab9345c70bbdc55fbc357c4eb8d28da64aa1aa Parents: 09e0817 Author: Stefan Richter Authored: Wed Dec 14 12:13:15 2016 +0100 Committer: Aljoscha Krettek Committed: Thu Dec 15 11:22:03 2016 +0100 ---------------------------------------------------------------------- .../api/checkpoint/ListCheckpointed.java | 1 + .../operators/AbstractUdfStreamOperator.java | 8 +- .../api/checkpoint/ListCheckpointedTest.java | 103 +++++++++++++++++++ .../StateInitializationContextImplTest.java | 21 ++++ 4 files changed, 130 insertions(+), 3 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/flink/blob/5dab9345/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/ListCheckpointed.java ---------------------------------------------------------------------- diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/ListCheckpointed.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/ListCheckpointed.java index 1031b88..5e85dc1 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/ListCheckpointed.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/ListCheckpointed.java @@ -41,6 +41,7 @@ public interface ListCheckpointed { * @param checkpointId The ID of the checkpoint. * @param timestamp Timestamp of the checkpoint. * @return The operator state in a list of redistributable, atomic sub-states. + * Should not return null, but empty list instead. * @throws Exception Thrown if the creation of the state object failed. This causes the * checkpoint to fail. The system may decide to fail the operation (and trigger * recovery), or to discard this checkpoint attempt and to continue running http://git-wip-us.apache.org/repos/asf/flink/blob/5dab9345/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java ---------------------------------------------------------------------- diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java index 1404958..81f709b 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java @@ -107,15 +107,17 @@ public abstract class AbstractUdfStreamOperator } else if (userFunction instanceof ListCheckpointed) { @SuppressWarnings("unchecked") List partitionableState = ((ListCheckpointed) userFunction). - snapshotState(context.getCheckpointId(), context.getCheckpointTimestamp()); + snapshotState(context.getCheckpointId(), context.getCheckpointTimestamp()); ListState listState = getOperatorStateBackend(). getSerializableListState(DefaultOperatorStateBackend.DEFAULT_OPERATOR_STATE_NAME); listState.clear(); - for (Serializable statePartition : partitionableState) { - listState.add(statePartition); + if (null != partitionableState) { + for (Serializable statePartition : partitionableState) { + listState.add(statePartition); + } } } http://git-wip-us.apache.org/repos/asf/flink/blob/5dab9345/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/checkpoint/ListCheckpointedTest.java ---------------------------------------------------------------------- diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/checkpoint/ListCheckpointedTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/checkpoint/ListCheckpointedTest.java new file mode 100644 index 0000000..6751617 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/checkpoint/ListCheckpointedTest.java @@ -0,0 +1,103 @@ +/* + * 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.flink.streaming.api.checkpoint; + +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.streaming.api.operators.StreamMap; +import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles; +import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +public class ListCheckpointedTest { + + @Test + public void testUDFReturningNull() throws Exception { + TestUserFunction userFunction = new TestUserFunction(null); + AbstractStreamOperatorTestHarness testHarness = + new AbstractStreamOperatorTestHarness<>(new StreamMap<>(userFunction), 1, 1, 0); + testHarness.open(); + OperatorStateHandles snapshot = testHarness.snapshot(0L, 0L); + testHarness.initializeState(snapshot); + Assert.assertTrue(userFunction.isRestored()); + } + + @Test + public void testUDFReturningEmpty() throws Exception { + TestUserFunction userFunction = new TestUserFunction(Collections.emptyList()); + AbstractStreamOperatorTestHarness testHarness = + new AbstractStreamOperatorTestHarness<>(new StreamMap<>(userFunction), 1, 1, 0); + testHarness.open(); + OperatorStateHandles snapshot = testHarness.snapshot(0L, 0L); + testHarness.initializeState(snapshot); + Assert.assertTrue(userFunction.isRestored()); + } + + @Test + public void testUDFReturningData() throws Exception { + TestUserFunction userFunction = new TestUserFunction(Arrays.asList(1, 2, 3)); + AbstractStreamOperatorTestHarness testHarness = + new AbstractStreamOperatorTestHarness<>(new StreamMap<>(userFunction), 1, 1, 0); + testHarness.open(); + OperatorStateHandles snapshot = testHarness.snapshot(0L, 0L); + testHarness.initializeState(snapshot); + Assert.assertTrue(userFunction.isRestored()); + } + + private static class TestUserFunction extends RichMapFunction implements ListCheckpointed { + + private static final long serialVersionUID = -8981369286399531925L; + + private final List expected; + private boolean restored; + + public TestUserFunction(List expected) { + this.expected = expected; + this.restored = false; + } + + @Override + public Integer map(Integer value) throws Exception { + return value; + } + + @Override + public List snapshotState(long checkpointId, long timestamp) throws Exception { + return expected; + } + + @Override + public void restoreState(List state) throws Exception { + if (null != expected) { + Assert.assertEquals(expected, state); + } else { + Assert.assertTrue(state.isEmpty()); + } + restored = true; + } + + public boolean isRestored() { + return restored; + } + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/flink/blob/5dab9345/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StateInitializationContextImplTest.java ---------------------------------------------------------------------- diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StateInitializationContextImplTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StateInitializationContextImplTest.java index cd94076..39dc5d6 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StateInitializationContextImplTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StateInitializationContextImplTest.java @@ -131,6 +131,27 @@ public class StateInitializationContextImplTest { @Test public void getOperatorStateStreams() throws Exception { + int i = 0; + int s = 0; + for (StatePartitionStreamProvider streamProvider : initializationContext.getRawOperatorStateInputs()) { + if (0 == i % 4) { + ++i; + } + Assert.assertNotNull(streamProvider); + try (InputStream is = streamProvider.getStream()) { + DataInputView div = new DataInputViewStreamWrapper(is); + + int val = div.readInt(); + Assert.assertEquals(i * NUM_HANDLES + s, val); + } + + ++s; + if (s == i % 4) { + s = 0; + ++i; + } + } + } @Test