You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by se...@apache.org on 2015/10/16 18:08:28 UTC
[01/24] flink git commit: [hotfix] Correct name of HDFS tests from
'org.apache.flink.tachyon' to 'org.apache.flink.hdfstests'
Repository: flink
Updated Branches:
refs/heads/master 6cb0fb51b -> 34c232e9b
[hotfix] Correct name of HDFS tests from 'org.apache.flink.tachyon' to 'org.apache.flink.hdfstests'
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/eec2d151
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/eec2d151
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/eec2d151
Branch: refs/heads/master
Commit: eec2d151165d26176f96535cad184e8f85dea318
Parents: 6cb0fb5
Author: Stephan Ewen <se...@apache.org>
Authored: Mon Oct 5 15:57:49 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 16 15:26:09 2015 +0200
----------------------------------------------------------------------
.../flink/hdfstests/FileStateHandleTest.java | 126 +++++++++++++
.../org/apache/flink/hdfstests/HDFSTest.java | 188 +++++++++++++++++++
.../flink/tachyon/FileStateHandleTest.java | 126 -------------
.../java/org/apache/flink/tachyon/HDFSTest.java | 188 -------------------
4 files changed, 314 insertions(+), 314 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/eec2d151/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/FileStateHandleTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/FileStateHandleTest.java b/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/FileStateHandleTest.java
new file mode 100644
index 0000000..59ee5a9
--- /dev/null
+++ b/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/FileStateHandleTest.java
@@ -0,0 +1,126 @@
+/*
+ * 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.hdfstests;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.Serializable;
+
+import org.apache.flink.runtime.state.FileStateHandle;
+import org.apache.flink.runtime.state.StateHandle;
+import org.apache.flink.runtime.state.StateHandleProvider;
+import org.apache.flink.util.SerializedValue;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class FileStateHandleTest {
+
+ private String hdfsURI;
+ private MiniDFSCluster hdfsCluster;
+ private org.apache.hadoop.fs.Path hdPath;
+ private org.apache.hadoop.fs.FileSystem hdfs;
+
+ @Before
+ public void createHDFS() {
+ try {
+ Configuration hdConf = new Configuration();
+
+ File baseDir = new File("./target/hdfs/filestatehandletest").getAbsoluteFile();
+ FileUtil.fullyDelete(baseDir);
+ hdConf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath());
+ MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(hdConf);
+ hdfsCluster = builder.build();
+
+ hdfsURI = "hdfs://" + hdfsCluster.getURI().getHost() + ":"
+ + hdfsCluster.getNameNodePort() + "/";
+
+ hdPath = new org.apache.hadoop.fs.Path("/StateHandleTest");
+ hdfs = hdPath.getFileSystem(hdConf);
+ hdfs.mkdirs(hdPath);
+
+ } catch (Throwable e) {
+ e.printStackTrace();
+ Assert.fail("Test failed " + e.getMessage());
+ }
+ }
+
+ @After
+ public void destroyHDFS() {
+ try {
+ hdfs.delete(hdPath, true);
+ hdfsCluster.shutdown();
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+
+ }
+
+ @Test
+ public void testFileStateHandle() throws Exception {
+
+ Serializable state = "state";
+
+ // Create a state handle provider for the hdfs directory
+ StateHandleProvider<Serializable> handleProvider = FileStateHandle.createProvider(hdfsURI
+ + hdPath);
+
+ FileStateHandle handle = (FileStateHandle) handleProvider.createStateHandle(state);
+
+ try {
+ handleProvider.createStateHandle(null);
+ fail();
+ } catch (RuntimeException e) {
+ // good
+ }
+
+ assertTrue(handle.stateFetched());
+ assertFalse(handle.isWritten());
+
+ // Serialize the handle so it writes the value to hdfs
+ SerializedValue<StateHandle<Serializable>> serializedHandle = new SerializedValue<StateHandle<Serializable>>(
+ handle);
+
+ assertTrue(handle.isWritten());
+
+ // Deserialize the handle and verify that the state is not fetched yet
+ FileStateHandle deserializedHandle = (FileStateHandle) serializedHandle
+ .deserializeValue(Thread.currentThread().getContextClassLoader());
+ assertFalse(deserializedHandle.stateFetched());
+
+ // Fetch the and compare with original
+ assertEquals(state, deserializedHandle.getState(this.getClass().getClassLoader()));
+
+ // Test whether discard removes the checkpoint file properly
+ assertTrue(hdfs.listFiles(hdPath, true).hasNext());
+ deserializedHandle.discardState();
+ assertFalse(hdfs.listFiles(hdPath, true).hasNext());
+
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/eec2d151/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/HDFSTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/HDFSTest.java b/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/HDFSTest.java
new file mode 100644
index 0000000..bc800a5
--- /dev/null
+++ b/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/HDFSTest.java
@@ -0,0 +1,188 @@
+/*
+ * 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.hdfstests;
+
+import org.apache.commons.io.IOUtils;
+import org.apache.flink.api.common.io.FileOutputFormat;
+import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.java.ExecutionEnvironmentFactory;
+import org.apache.flink.api.java.LocalEnvironment;
+import org.apache.flink.api.java.io.AvroOutputFormat;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.examples.java.wordcount.WordCount;
+import org.apache.flink.runtime.fs.hdfs.HadoopFileSystem;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.StringWriter;
+
+/**
+ * This test should logically be located in the 'flink-runtime' tests. However, this project
+ * has already all dependencies required (flink-java-examples). Also, the ParallelismOneExecEnv is here.
+ */
+public class HDFSTest {
+
+ protected String hdfsURI;
+ private MiniDFSCluster hdfsCluster;
+ private org.apache.hadoop.fs.Path hdPath;
+ protected org.apache.hadoop.fs.FileSystem hdfs;
+
+ @Before
+ public void createHDFS() {
+ try {
+ Configuration hdConf = new Configuration();
+
+ File baseDir = new File("./target/hdfs/hdfsTest").getAbsoluteFile();
+ FileUtil.fullyDelete(baseDir);
+ hdConf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath());
+ MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(hdConf);
+ hdfsCluster = builder.build();
+
+ hdfsURI = "hdfs://" + hdfsCluster.getURI().getHost() + ":" + hdfsCluster.getNameNodePort() +"/";
+
+ hdPath = new org.apache.hadoop.fs.Path("/test");
+ hdfs = hdPath.getFileSystem(hdConf);
+ FSDataOutputStream stream = hdfs.create(hdPath);
+ for(int i = 0; i < 10; i++) {
+ stream.write("Hello HDFS\n".getBytes());
+ }
+ stream.close();
+
+ } catch(Throwable e) {
+ e.printStackTrace();
+ Assert.fail("Test failed " + e.getMessage());
+ }
+ }
+
+ @After
+ public void destroyHDFS() {
+ try {
+ hdfs.delete(hdPath, false);
+ hdfsCluster.shutdown();
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+
+ }
+
+ @Test
+ public void testHDFS() {
+
+ Path file = new Path(hdfsURI + hdPath);
+ org.apache.hadoop.fs.Path result = new org.apache.hadoop.fs.Path(hdfsURI + "/result");
+ try {
+ FileSystem fs = file.getFileSystem();
+ Assert.assertTrue("Must be HadoopFileSystem", fs instanceof HadoopFileSystem);
+
+ DopOneTestEnvironment.setAsContext();
+ try {
+ WordCount.main(new String[]{file.toString(), result.toString()});
+ }
+ catch(Throwable t) {
+ t.printStackTrace();
+ Assert.fail("Test failed with " + t.getMessage());
+ }
+ finally {
+ DopOneTestEnvironment.unsetAsContext();
+ }
+
+ Assert.assertTrue("No result file present", hdfs.exists(result));
+
+ // validate output:
+ org.apache.hadoop.fs.FSDataInputStream inStream = hdfs.open(result);
+ StringWriter writer = new StringWriter();
+ IOUtils.copy(inStream, writer);
+ String resultString = writer.toString();
+
+ Assert.assertEquals("hdfs 10\n" +
+ "hello 10\n", resultString);
+ inStream.close();
+
+ } catch (IOException e) {
+ e.printStackTrace();
+ Assert.fail("Error in test: " + e.getMessage() );
+ }
+ }
+
+ @Test
+ public void testAvroOut() {
+ String type = "one";
+ AvroOutputFormat<String> avroOut =
+ new AvroOutputFormat<String>( String.class );
+
+ org.apache.hadoop.fs.Path result = new org.apache.hadoop.fs.Path(hdfsURI + "/avroTest");
+
+ avroOut.setOutputFilePath(new Path(result.toString()));
+ avroOut.setWriteMode(FileSystem.WriteMode.NO_OVERWRITE);
+ avroOut.setOutputDirectoryMode(FileOutputFormat.OutputDirectoryMode.ALWAYS);
+
+ try {
+ avroOut.open(0, 2);
+ avroOut.writeRecord(type);
+ avroOut.close();
+
+ avroOut.open(1, 2);
+ avroOut.writeRecord(type);
+ avroOut.close();
+
+
+ Assert.assertTrue("No result file present", hdfs.exists(result));
+ FileStatus[] files = hdfs.listStatus(result);
+ Assert.assertEquals(2, files.length);
+ for(FileStatus file : files) {
+ Assert.assertTrue("1.avro".equals(file.getPath().getName()) || "2.avro".equals(file.getPath().getName()));
+ }
+
+ } catch (IOException e) {
+ e.printStackTrace();
+ Assert.fail(e.getMessage());
+ }
+ }
+
+ // package visible
+ static abstract class DopOneTestEnvironment extends ExecutionEnvironment {
+
+ public static void setAsContext() {
+ final LocalEnvironment le = new LocalEnvironment();
+ le.setParallelism(1);
+
+ initializeContextEnvironment(new ExecutionEnvironmentFactory() {
+
+ @Override
+ public ExecutionEnvironment createExecutionEnvironment() {
+ return le;
+ }
+ });
+ }
+
+ public static void unsetAsContext() {
+ resetContextEnvironment();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/eec2d151/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/tachyon/FileStateHandleTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/tachyon/FileStateHandleTest.java b/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/tachyon/FileStateHandleTest.java
deleted file mode 100644
index a8734e6..0000000
--- a/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/tachyon/FileStateHandleTest.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/*
- * 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.tachyon;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.Serializable;
-
-import org.apache.flink.runtime.state.FileStateHandle;
-import org.apache.flink.runtime.state.StateHandle;
-import org.apache.flink.runtime.state.StateHandleProvider;
-import org.apache.flink.util.SerializedValue;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-public class FileStateHandleTest {
-
- private String hdfsURI;
- private MiniDFSCluster hdfsCluster;
- private org.apache.hadoop.fs.Path hdPath;
- private org.apache.hadoop.fs.FileSystem hdfs;
-
- @Before
- public void createHDFS() {
- try {
- Configuration hdConf = new Configuration();
-
- File baseDir = new File("./target/hdfs/filestatehandletest").getAbsoluteFile();
- FileUtil.fullyDelete(baseDir);
- hdConf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath());
- MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(hdConf);
- hdfsCluster = builder.build();
-
- hdfsURI = "hdfs://" + hdfsCluster.getURI().getHost() + ":"
- + hdfsCluster.getNameNodePort() + "/";
-
- hdPath = new org.apache.hadoop.fs.Path("/StateHandleTest");
- hdfs = hdPath.getFileSystem(hdConf);
- hdfs.mkdirs(hdPath);
-
- } catch (Throwable e) {
- e.printStackTrace();
- Assert.fail("Test failed " + e.getMessage());
- }
- }
-
- @After
- public void destroyHDFS() {
- try {
- hdfs.delete(hdPath, true);
- hdfsCluster.shutdown();
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
-
- }
-
- @Test
- public void testFileStateHandle() throws Exception {
-
- Serializable state = "state";
-
- // Create a state handle provider for the hdfs directory
- StateHandleProvider<Serializable> handleProvider = FileStateHandle.createProvider(hdfsURI
- + hdPath);
-
- FileStateHandle handle = (FileStateHandle) handleProvider.createStateHandle(state);
-
- try {
- handleProvider.createStateHandle(null);
- fail();
- } catch (RuntimeException e) {
- // good
- }
-
- assertTrue(handle.stateFetched());
- assertFalse(handle.isWritten());
-
- // Serialize the handle so it writes the value to hdfs
- SerializedValue<StateHandle<Serializable>> serializedHandle = new SerializedValue<StateHandle<Serializable>>(
- handle);
-
- assertTrue(handle.isWritten());
-
- // Deserialize the handle and verify that the state is not fetched yet
- FileStateHandle deserializedHandle = (FileStateHandle) serializedHandle
- .deserializeValue(Thread.currentThread().getContextClassLoader());
- assertFalse(deserializedHandle.stateFetched());
-
- // Fetch the and compare with original
- assertEquals(state, deserializedHandle.getState(this.getClass().getClassLoader()));
-
- // Test whether discard removes the checkpoint file properly
- assertTrue(hdfs.listFiles(hdPath, true).hasNext());
- deserializedHandle.discardState();
- assertFalse(hdfs.listFiles(hdPath, true).hasNext());
-
- }
-
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/eec2d151/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/tachyon/HDFSTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/tachyon/HDFSTest.java b/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/tachyon/HDFSTest.java
deleted file mode 100644
index 5ec0add..0000000
--- a/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/tachyon/HDFSTest.java
+++ /dev/null
@@ -1,188 +0,0 @@
-/*
- * 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.tachyon;
-
-import org.apache.commons.io.IOUtils;
-import org.apache.flink.api.common.io.FileOutputFormat;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.ExecutionEnvironmentFactory;
-import org.apache.flink.api.java.LocalEnvironment;
-import org.apache.flink.api.java.io.AvroOutputFormat;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.examples.java.wordcount.WordCount;
-import org.apache.flink.runtime.fs.hdfs.HadoopFileSystem;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.StringWriter;
-
-/**
- * This test should logically be located in the 'flink-runtime' tests. However, this project
- * has already all dependencies required (flink-java-examples). Also, the ParallelismOneExecEnv is here.
- */
-public class HDFSTest {
-
- protected String hdfsURI;
- private MiniDFSCluster hdfsCluster;
- private org.apache.hadoop.fs.Path hdPath;
- protected org.apache.hadoop.fs.FileSystem hdfs;
-
- @Before
- public void createHDFS() {
- try {
- Configuration hdConf = new Configuration();
-
- File baseDir = new File("./target/hdfs/hdfsTest").getAbsoluteFile();
- FileUtil.fullyDelete(baseDir);
- hdConf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath());
- MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(hdConf);
- hdfsCluster = builder.build();
-
- hdfsURI = "hdfs://" + hdfsCluster.getURI().getHost() + ":" + hdfsCluster.getNameNodePort() +"/";
-
- hdPath = new org.apache.hadoop.fs.Path("/test");
- hdfs = hdPath.getFileSystem(hdConf);
- FSDataOutputStream stream = hdfs.create(hdPath);
- for(int i = 0; i < 10; i++) {
- stream.write("Hello HDFS\n".getBytes());
- }
- stream.close();
-
- } catch(Throwable e) {
- e.printStackTrace();
- Assert.fail("Test failed " + e.getMessage());
- }
- }
-
- @After
- public void destroyHDFS() {
- try {
- hdfs.delete(hdPath, false);
- hdfsCluster.shutdown();
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
-
- }
-
- @Test
- public void testHDFS() {
-
- Path file = new Path(hdfsURI + hdPath);
- org.apache.hadoop.fs.Path result = new org.apache.hadoop.fs.Path(hdfsURI + "/result");
- try {
- FileSystem fs = file.getFileSystem();
- Assert.assertTrue("Must be HadoopFileSystem", fs instanceof HadoopFileSystem);
-
- DopOneTestEnvironment.setAsContext();
- try {
- WordCount.main(new String[]{file.toString(), result.toString()});
- }
- catch(Throwable t) {
- t.printStackTrace();
- Assert.fail("Test failed with " + t.getMessage());
- }
- finally {
- DopOneTestEnvironment.unsetAsContext();
- }
-
- Assert.assertTrue("No result file present", hdfs.exists(result));
-
- // validate output:
- org.apache.hadoop.fs.FSDataInputStream inStream = hdfs.open(result);
- StringWriter writer = new StringWriter();
- IOUtils.copy(inStream, writer);
- String resultString = writer.toString();
-
- Assert.assertEquals("hdfs 10\n" +
- "hello 10\n", resultString);
- inStream.close();
-
- } catch (IOException e) {
- e.printStackTrace();
- Assert.fail("Error in test: " + e.getMessage() );
- }
- }
-
- @Test
- public void testAvroOut() {
- String type = "one";
- AvroOutputFormat<String> avroOut =
- new AvroOutputFormat<String>( String.class );
-
- org.apache.hadoop.fs.Path result = new org.apache.hadoop.fs.Path(hdfsURI + "/avroTest");
-
- avroOut.setOutputFilePath(new Path(result.toString()));
- avroOut.setWriteMode(FileSystem.WriteMode.NO_OVERWRITE);
- avroOut.setOutputDirectoryMode(FileOutputFormat.OutputDirectoryMode.ALWAYS);
-
- try {
- avroOut.open(0, 2);
- avroOut.writeRecord(type);
- avroOut.close();
-
- avroOut.open(1, 2);
- avroOut.writeRecord(type);
- avroOut.close();
-
-
- Assert.assertTrue("No result file present", hdfs.exists(result));
- FileStatus[] files = hdfs.listStatus(result);
- Assert.assertEquals(2, files.length);
- for(FileStatus file : files) {
- Assert.assertTrue("1.avro".equals(file.getPath().getName()) || "2.avro".equals(file.getPath().getName()));
- }
-
- } catch (IOException e) {
- e.printStackTrace();
- Assert.fail(e.getMessage());
- }
- }
-
- // package visible
- static abstract class DopOneTestEnvironment extends ExecutionEnvironment {
-
- public static void setAsContext() {
- final LocalEnvironment le = new LocalEnvironment();
- le.setParallelism(1);
-
- initializeContextEnvironment(new ExecutionEnvironmentFactory() {
-
- @Override
- public ExecutionEnvironment createExecutionEnvironment() {
- return le;
- }
- });
- }
-
- public static void unsetAsContext() {
- resetContextEnvironment();
- }
- }
-}
[09/24] flink git commit: [FLINK-2808] [streaming] Refactor and
extend state backend abstraction
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java
index cf3dcfc..9718b72 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java
@@ -28,9 +28,8 @@ import org.apache.flink.api.common.cache.DistributedCache;
import org.apache.flink.api.common.functions.BroadcastVariableInitializer;
import org.apache.flink.api.common.functions.RuntimeContext;
import org.apache.flink.api.common.state.OperatorState;
-import org.apache.flink.api.common.state.StateCheckpointer;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
-import java.io.IOException;
import java.io.Serializable;
import java.util.List;
import java.util.Map;
@@ -63,69 +62,71 @@ public class MockRuntimeContext implements RuntimeContext {
@Override
public ExecutionConfig getExecutionConfig() {
- return null;
+ throw new UnsupportedOperationException();
}
@Override
public ClassLoader getUserCodeClassLoader() {
- return null;
+ throw new UnsupportedOperationException();
}
@Override
- public <V, A extends Serializable> void addAccumulator(String name, Accumulator<V, A> accumulator) {}
+ public <V, A extends Serializable> void addAccumulator(String name, Accumulator<V, A> accumulator) {
+ throw new UnsupportedOperationException();
+ }
@Override
public <V, A extends Serializable> Accumulator<V, A> getAccumulator(String name) {
- return null;
+ throw new UnsupportedOperationException();
}
@Override
public Map<String, Accumulator<?, ?>> getAllAccumulators() {
- return null;
+ throw new UnsupportedOperationException();
}
@Override
public IntCounter getIntCounter(String name) {
- return null;
+ throw new UnsupportedOperationException();
}
@Override
public LongCounter getLongCounter(String name) {
- return null;
+ throw new UnsupportedOperationException();
}
@Override
public DoubleCounter getDoubleCounter(String name) {
- return null;
+ throw new UnsupportedOperationException();
}
@Override
public Histogram getHistogram(String name) {
- return null;
+ throw new UnsupportedOperationException();
}
@Override
public <RT> List<RT> getBroadcastVariable(String name) {
- return null;
+ throw new UnsupportedOperationException();
}
@Override
public <T, C> C getBroadcastVariableWithInitializer(String name, BroadcastVariableInitializer<T, C> initializer) {
- return null;
+ throw new UnsupportedOperationException();
}
@Override
public DistributedCache getDistributedCache() {
- return null;
+ throw new UnsupportedOperationException();
}
@Override
- public <S, C extends Serializable> OperatorState<S> getOperatorState(String name, S defaultState, boolean partitioned, StateCheckpointer<S, C> checkpointer) throws IOException {
- return null;
+ public <S> OperatorState<S> getKeyValueState(Class<S> stateType, S defaultState) {
+ throw new UnsupportedOperationException();
}
@Override
- public <S extends Serializable> OperatorState<S> getOperatorState(String name, S defaultState, boolean partitioned) throws IOException {
- return null;
+ public <S> OperatorState<S> getKeyValueState(TypeInformation<S> stateType, S defaultState) {
+ throw new UnsupportedOperationException();
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/BroadcastOutputSelectorWrapper.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/BroadcastOutputSelectorWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/BroadcastOutputSelectorWrapper.java
index 00c6f80..7034b11 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/BroadcastOutputSelectorWrapper.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/BroadcastOutputSelectorWrapper.java
@@ -18,7 +18,6 @@
package org.apache.flink.streaming.api.collector.selector;
import java.util.ArrayList;
-import java.util.List;
import org.apache.flink.streaming.api.graph.StreamEdge;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
@@ -27,17 +26,16 @@ import org.apache.flink.util.Collector;
public class BroadcastOutputSelectorWrapper<OUT> implements OutputSelectorWrapper<OUT> {
private static final long serialVersionUID = 1L;
- private List<Collector<StreamRecord<OUT>>> outputs;
+
+ private final ArrayList<Collector<StreamRecord<OUT>>> outputs;
public BroadcastOutputSelectorWrapper() {
outputs = new ArrayList<Collector<StreamRecord<OUT>>>();
}
-
- @SuppressWarnings("unchecked,rawtypes")
+
@Override
- public void addCollector(Collector<StreamRecord<?>> output, StreamEdge edge) {
- Collector output1 = output;
- outputs.add((Collector<StreamRecord<OUT>>) output1);
+ public void addCollector(Collector<StreamRecord<OUT>> output, StreamEdge edge) {
+ outputs.add(output);
}
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.java
index c6e3388..84558fc 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/DirectedOutputSelectorWrapper.java
@@ -17,16 +17,16 @@
package org.apache.flink.streaming.api.collector.selector;
+import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
-import java.util.LinkedList;
import java.util.List;
-import java.util.Map;
import java.util.Set;
import org.apache.flink.streaming.api.graph.StreamEdge;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.util.Collector;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -38,32 +38,31 @@ public class DirectedOutputSelectorWrapper<OUT> implements OutputSelectorWrapper
private List<OutputSelector<OUT>> outputSelectors;
- private Map<String, List<Collector<StreamRecord<OUT>>>> outputMap;
- private Set<Collector<StreamRecord<OUT>>> selectAllOutputs;
+ private HashMap<String, ArrayList<Collector<StreamRecord<OUT>>>> outputMap;
+ private HashSet<Collector<StreamRecord<OUT>>> selectAllOutputs;
public DirectedOutputSelectorWrapper(List<OutputSelector<OUT>> outputSelectors) {
this.outputSelectors = outputSelectors;
- this.selectAllOutputs = new HashSet<Collector<StreamRecord<OUT>>>(); //new LinkedList<Collector<OUT>>();
- this.outputMap = new HashMap<String, List<Collector<StreamRecord<OUT>>>>();
+ this.selectAllOutputs = new HashSet<Collector<StreamRecord<OUT>>>();
+ this.outputMap = new HashMap<String, ArrayList<Collector<StreamRecord<OUT>>>>();
}
-
- @SuppressWarnings("unchecked,rawtypes")
+
@Override
- public void addCollector(Collector<StreamRecord<?>> output, StreamEdge edge) {
- Collector output1 = output;
+ public void addCollector(Collector<StreamRecord<OUT>> output, StreamEdge edge) {
List<String> selectedNames = edge.getSelectedNames();
if (selectedNames.isEmpty()) {
- selectAllOutputs.add((Collector<StreamRecord<OUT>>) output1);
- } else {
+ selectAllOutputs.add(output);
+ }
+ else {
for (String selectedName : selectedNames) {
-
if (!outputMap.containsKey(selectedName)) {
- outputMap.put(selectedName, new LinkedList<Collector<StreamRecord<OUT>>>());
- outputMap.get(selectedName).add((Collector<StreamRecord<OUT>>) output1);
- } else {
+ outputMap.put(selectedName, new ArrayList<Collector<StreamRecord<OUT>>>());
+ outputMap.get(selectedName).add(output);
+ }
+ else {
if (!outputMap.get(selectedName).contains(output)) {
- outputMap.get(selectedName).add((Collector<StreamRecord<OUT>>) output1);
+ outputMap.get(selectedName).add(output);
}
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapper.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapper.java
index 9133ac0..f25c995 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapper.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/collector/selector/OutputSelectorWrapper.java
@@ -25,7 +25,7 @@ import org.apache.flink.util.Collector;
public interface OutputSelectorWrapper<OUT> extends Serializable {
- public void addCollector(Collector<StreamRecord<?>> output, StreamEdge edge);
+ public void addCollector(Collector<StreamRecord<OUT>> output, StreamEdge edge);
public Iterable<Collector<StreamRecord<OUT>>> getSelectedOutputs(OUT record);
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index 00991a7..7e686c7 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -17,10 +17,10 @@
package org.apache.flink.streaming.api.datastream;
+import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
-import com.google.common.collect.Lists;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.functions.FilterFunction;
import org.apache.flink.api.common.functions.FlatMapFunction;
@@ -172,8 +172,9 @@ public class DataStream<T> {
* The DataStreams to union output with.
* @return The {@link DataStream}.
*/
- public DataStream<T> union(DataStream<T>... streams) {
- List<StreamTransformation<T>> unionedTransforms = Lists.newArrayList();
+ @SafeVarargs
+ public final DataStream<T> union(DataStream<T>... streams) {
+ List<StreamTransformation<T>> unionedTransforms = new ArrayList<>();
unionedTransforms.add(this.transformation);
Collection<StreamTransformation<?>> thisPredecessors = this.getTransformation().getTransitivePredecessors();
@@ -185,6 +186,11 @@ public class DataStream<T> {
"This Stream: " + this.getTransformation() +
", other stream: " + newStream.getTransformation());
}
+ if (!getType().equals(newStream.getType())) {
+ throw new IllegalArgumentException("Cannot union streams of different types: "
+ + getType() + " and " + newStream.getType());
+ }
+
Collection<StreamTransformation<?>> predecessors = newStream.getTransformation().getTransitivePredecessors();
if (predecessors.contains(this.transformation) || thisPredecessors.contains(newStream.getTransformation())) {
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
index fdf398c..24104ad 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
@@ -17,7 +17,7 @@
package org.apache.flink.streaming.api.datastream;
-import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
import org.apache.flink.streaming.api.operators.StreamSink;
import org.apache.flink.streaming.api.transformations.SinkTransformation;
@@ -76,7 +76,7 @@ public class DataStreamSink<T> {
* @return The sink with chaining disabled
*/
public DataStreamSink<T> disableChaining() {
- this.transformation.setChainingStrategy(AbstractStreamOperator.ChainingStrategy.NEVER);
+ this.transformation.setChainingStrategy(ChainingStrategy.NEVER);
return this;
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java
index 0da419c..cdea910 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java
@@ -63,6 +63,8 @@ public class KeyedStream<T, KEY> extends DataStream<T> {
protected final KeySelector<T, KEY> keySelector;
+ protected final TypeInformation<KEY> keyType;
+
/**
* Creates a new {@link KeyedStream} using the given {@link KeySelector}
* to partition operator state by key.
@@ -73,8 +75,23 @@ public class KeyedStream<T, KEY> extends DataStream<T> {
* Function for determining state partitions
*/
public KeyedStream(DataStream<T> dataStream, KeySelector<T, KEY> keySelector) {
- super(dataStream.getExecutionEnvironment(), new PartitionTransformation<>(dataStream.getTransformation(), new HashPartitioner<>(keySelector)));
+ this(dataStream, keySelector, TypeExtractor.getKeySelectorTypes(keySelector, dataStream.getType()));
+ }
+
+ /**
+ * Creates a new {@link KeyedStream} using the given {@link KeySelector}
+ * to partition operator state by key.
+ *
+ * @param dataStream
+ * Base stream of data
+ * @param keySelector
+ * Function for determining state partitions
+ */
+ public KeyedStream(DataStream<T> dataStream, KeySelector<T, KEY> keySelector, TypeInformation<KEY> keyType) {
+ super(dataStream.getExecutionEnvironment(), new PartitionTransformation<>(
+ dataStream.getTransformation(), new HashPartitioner<>(keySelector)));
this.keySelector = keySelector;
+ this.keyType = keyType;
}
@@ -95,7 +112,11 @@ public class KeyedStream<T, KEY> extends DataStream<T> {
SingleOutputStreamOperator<R, ?> returnStream = super.transform(operatorName, outTypeInfo,operator);
- ((OneInputTransformation<T, R>) returnStream.getTransformation()).setStateKeySelector(keySelector);
+ // inject the key selector and key type
+ OneInputTransformation<T, R> transform = (OneInputTransformation<T, R>) returnStream.getTransformation();
+ transform.setStateKeySelector(keySelector);
+ transform.setStateKeyType(keyType);
+
return returnStream;
}
@@ -105,6 +126,7 @@ public class KeyedStream<T, KEY> extends DataStream<T> {
public DataStreamSink<T> addSink(SinkFunction<T> sinkFunction) {
DataStreamSink<T> result = super.addSink(sinkFunction);
result.getTransformation().setStateKeySelector(keySelector);
+ result.getTransformation().setStateKeyType(keyType);
return result;
}
@@ -197,7 +219,7 @@ public class KeyedStream<T, KEY> extends DataStream<T> {
*/
public SingleOutputStreamOperator<T, ?> reduce(ReduceFunction<T> reducer) {
return transform("Keyed Reduce", getType(), new StreamGroupedReduce<T>(
- clean(reducer), keySelector, getType()));
+ clean(reducer), getType().createSerializer(getExecutionConfig())));
}
/**
@@ -215,11 +237,10 @@ public class KeyedStream<T, KEY> extends DataStream<T> {
*/
public <R> SingleOutputStreamOperator<R, ?> fold(R initialValue, FoldFunction<T, R> folder) {
- TypeInformation<R> outType = TypeExtractor.getFoldReturnTypes(clean(folder), getType(),
- Utils.getCallLocationName(), true);
+ TypeInformation<R> outType = TypeExtractor.getFoldReturnTypes(
+ clean(folder), getType(), Utils.getCallLocationName(), true);
- return transform("Keyed Fold", outType, new StreamGroupedFold<>(clean(folder),
- keySelector, initialValue, getType()));
+ return transform("Keyed Fold", outType, new StreamGroupedFold<>(clean(folder), initialValue));
}
/**
@@ -454,7 +475,8 @@ public class KeyedStream<T, KEY> extends DataStream<T> {
}
protected SingleOutputStreamOperator<T, ?> aggregate(AggregationFunction<T> aggregate) {
- StreamGroupedReduce<T> operator = new StreamGroupedReduce<T>(clean(aggregate), keySelector, getType());
+ StreamGroupedReduce<T> operator = new StreamGroupedReduce<T>(
+ clean(aggregate), getType().createSerializer(getExecutionConfig()));
return transform("Keyed Aggregation", getType(), operator);
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
index 016cf5e..33d5a3c 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/SingleOutputStreamOperator.java
@@ -23,8 +23,7 @@ import org.apache.flink.api.java.typeutils.TypeExtractor;
import org.apache.flink.api.java.typeutils.TypeInfoParser;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.graph.StreamGraph.ResourceStrategy;
-import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
-import org.apache.flink.streaming.api.operators.StreamOperator.ChainingStrategy;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
import org.apache.flink.streaming.api.transformations.PartitionTransformation;
import org.apache.flink.streaming.api.transformations.StreamTransformation;
import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
@@ -141,7 +140,7 @@ public class SingleOutputStreamOperator<T, O extends SingleOutputStreamOperator<
* @return The operator with chaining disabled
*/
public SingleOutputStreamOperator<T, O> disableChaining() {
- return setChainingStrategy(AbstractStreamOperator.ChainingStrategy.NEVER);
+ return setChainingStrategy(ChainingStrategy.NEVER);
}
/**
@@ -152,7 +151,7 @@ public class SingleOutputStreamOperator<T, O extends SingleOutputStreamOperator<
* @return The operator with chaining set.
*/
public SingleOutputStreamOperator<T, O> startNewChain() {
- return setChainingStrategy(AbstractStreamOperator.ChainingStrategy.HEAD);
+ return setChainingStrategy(ChainingStrategy.HEAD);
}
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
index 28410fd..26e1c9e 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
@@ -45,8 +45,6 @@ import org.apache.flink.client.program.OptimizerPlanEnvironment;
import org.apache.flink.client.program.PreviewPlanEnvironment;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.fs.Path;
-import org.apache.flink.runtime.state.FileStateHandle;
-import org.apache.flink.runtime.state.StateHandleProvider;
import org.apache.flink.streaming.api.CheckpointingMode;
import org.apache.flink.streaming.api.TimeCharacteristic;
import org.apache.flink.streaming.api.datastream.DataStream;
@@ -65,6 +63,7 @@ import org.apache.flink.streaming.api.functions.source.StatefulSequenceSource;
import org.apache.flink.streaming.api.graph.StreamGraph;
import org.apache.flink.streaming.api.graph.StreamGraphGenerator;
import org.apache.flink.streaming.api.operators.StreamSource;
+import org.apache.flink.streaming.api.state.StateBackend;
import org.apache.flink.streaming.api.transformations.StreamTransformation;
import org.apache.flink.types.StringValue;
import org.apache.flink.util.SplittableIterator;
@@ -77,7 +76,8 @@ import java.util.Arrays;
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
-import java.util.Objects;
+
+import static java.util.Objects.requireNonNull;
/**
* An ExecutionEnvironment for streaming jobs. An instance of it is
@@ -127,8 +127,9 @@ public abstract class StreamExecutionEnvironment {
protected CheckpointingMode checkpointingMode;
protected boolean forceCheckpointing = false;
-
- protected StateHandleProvider<?> stateHandleProvider;
+
+ /** The state backend used for storing k/v state and state snapshots */
+ private StateBackend<?> defaultStateBackend;
/** The time characteristic used by the data streams */
private TimeCharacteristic timeCharacteristic = DEFAULT_TIME_CHARACTERISTIC;
@@ -155,8 +156,7 @@ public abstract class StreamExecutionEnvironment {
* program via the command line client from a JAR file, the default degree
* of parallelism is the one configured for that setup.
*
- * @param parallelism
- * The parallelism
+ * @param parallelism The parallelism
*/
public StreamExecutionEnvironment setParallelism(int parallelism) {
if (parallelism < 1) {
@@ -365,27 +365,40 @@ public abstract class StreamExecutionEnvironment {
}
/**
- * Sets the {@link StateHandleProvider} used for storing operator state
- * checkpoints when checkpointing is enabled.
- * <p>
- * An example would be using a {@link FileStateHandle#createProvider(String)}
- * to use any Flink supported file system as a state backend
+ * Sets the state backend that describes how to store and checkpoint operator state. It defines in
+ * what form the key/value state ({@link org.apache.flink.api.common.state.OperatorState}, accessible
+ * from operations on {@link org.apache.flink.streaming.api.datastream.KeyedStream}) is maintained
+ * (heap, managed memory, externally), and where state snapshots/checkpoints are stored, both for
+ * the key/value state, and for checkpointed functions (implementing the interface
+ * {@link org.apache.flink.streaming.api.checkpoint.Checkpointed}).
+ *
+ * <p>The {@link org.apache.flink.streaming.api.state.memory.MemoryStateBackend} for example
+ * maintains the state in heap memory, as objects. It is lightweight without extra dependencies,
+ * but can checkpoint only small states (some counters).
*
+ * <p>In contrast, the {@link org.apache.flink.streaming.api.state.filesystem.FsStateBackend}
+ * stores checkpoints of the state (also maintained as heap objects) in files. When using a replicated
+ * file system (like HDFS, S3, MapR FS, Tachyon, etc) this will guarantee that state is not lost upon
+ * failures of individual nodes and that streaming program can be executed highly available and strongly
+ * consistent (assuming that Flink is run in high-availability mode).
+ *
+ * @return This StreamExecutionEnvironment itself, to allow chaining of function calls.
+ *
+ * @see #getStateBackend()
*/
- public StreamExecutionEnvironment setStateHandleProvider(StateHandleProvider<?> provider) {
- this.stateHandleProvider = provider;
+ public StreamExecutionEnvironment setStateBackend(StateBackend<?> backend) {
+ this.defaultStateBackend = requireNonNull(backend);
return this;
}
/**
- * Returns the {@link org.apache.flink.runtime.state.StateHandle}
- *
- * @see #setStateHandleProvider(org.apache.flink.runtime.state.StateHandleProvider)
- *
- * @return The StateHandleProvider
+ * Returns the state backend that defines how to store and checkpoint state.
+ * @return The state backend that defines how to store and checkpoint state.
+ *
+ * @see #setStateBackend(StateBackend)
*/
- public StateHandleProvider<?> getStateHandleProvider() {
- return stateHandleProvider;
+ public StateBackend<?> getStateBackend() {
+ return defaultStateBackend;
}
/**
@@ -395,8 +408,7 @@ public abstract class StreamExecutionEnvironment {
* should be used.
*
* @param numberOfExecutionRetries
- * The number of times the system will try to re-execute failed
- * tasks.
+ * The number of times the system will try to re-execute failed tasks.
*/
public void setNumberOfExecutionRetries(int numberOfExecutionRetries) {
config.setNumberOfExecutionRetries(numberOfExecutionRetries);
@@ -423,7 +435,7 @@ public abstract class StreamExecutionEnvironment {
* The delay of time the system will wait to re-execute failed
* tasks.
*/
- public void setExecutionRetryDelay(long executionRetryDelay){
+ public void setExecutionRetryDelay(long executionRetryDelay) {
config.setExecutionRetryDelay(executionRetryDelay);
}
@@ -434,7 +446,7 @@ public abstract class StreamExecutionEnvironment {
*
* @return The delay time the system will wait to re-execute failed tasks.
*/
- public long getExecutionRetryDelay(){
+ public long getExecutionRetryDelay() {
return config.getExecutionRetryDelay();
}
/**
@@ -550,7 +562,7 @@ public abstract class StreamExecutionEnvironment {
* @param characteristic The time characteristic.
*/
public void setStreamTimeCharacteristic(TimeCharacteristic characteristic) {
- this.timeCharacteristic = Objects.requireNonNull(characteristic);
+ this.timeCharacteristic = requireNonNull(characteristic);
if (characteristic == TimeCharacteristic.ProcessingTime) {
getConfig().disableTimestamps();
getConfig().setAutoWatermarkInterval(0);
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/FileSinkFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/FileSinkFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/FileSinkFunction.java
index 1cf5c07..504bc39 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/FileSinkFunction.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/FileSinkFunction.java
@@ -20,10 +20,11 @@ package org.apache.flink.streaming.api.functions.sink;
import java.io.IOException;
import java.util.ArrayList;
+import org.apache.flink.api.common.functions.RuntimeContext;
import org.apache.flink.api.common.io.CleanupWhenUnsuccessful;
import org.apache.flink.api.common.io.OutputFormat;
import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -37,8 +38,11 @@ import org.slf4j.LoggerFactory;
* Input type
*/
public abstract class FileSinkFunction<IN> extends RichSinkFunction<IN> {
+
private static final long serialVersionUID = 1L;
+
private static final Logger LOG = LoggerFactory.getLogger(FileSinkFunction.class);
+
protected ArrayList<IN> tupleList = new ArrayList<IN>();
protected volatile OutputFormat<IN> format;
protected volatile boolean cleanupCalled = false;
@@ -51,8 +55,8 @@ public abstract class FileSinkFunction<IN> extends RichSinkFunction<IN> {
@Override
public void open(Configuration parameters) throws Exception {
- StreamingRuntimeContext context = (StreamingRuntimeContext) getRuntimeContext();
- format.configure(context.getTaskStubParameters());
+ RuntimeContext context = getRuntimeContext();
+ format.configure(parameters);
indexInSubtaskGroup = context.getIndexOfThisSubtask();
currentNumberOfSubtasks = context.getNumberOfParallelSubtasks();
format.open(indexInSubtaskGroup, currentNumberOfSubtasks);
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/PrintSinkFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/PrintSinkFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/PrintSinkFunction.java
index 5a9c7a8..93a91cd 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/PrintSinkFunction.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/sink/PrintSinkFunction.java
@@ -20,7 +20,7 @@ package org.apache.flink.streaming.api.functions.sink;
import java.io.PrintStream;
import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
/**
* Implementation of the SinkFunction writing every tuple to the standard
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FileSourceFunction.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FileSourceFunction.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FileSourceFunction.java
index 253c076..cc3925c 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FileSourceFunction.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/FileSourceFunction.java
@@ -23,7 +23,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.io.InputSplit;
import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
-import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
import java.util.Iterator;
import java.util.NoSuchElementException;
@@ -52,7 +52,8 @@ public class FileSourceFunction<OUT> extends RichParallelSourceFunction<OUT> {
public void open(Configuration parameters) throws Exception {
StreamingRuntimeContext context = (StreamingRuntimeContext) getRuntimeContext();
this.provider = context.getInputSplitProvider();
- format.configure(context.getTaskStubParameters());
+
+ format.configure(parameters);
serializer = typeInfo.createSerializer(getRuntimeContext().getExecutionConfig());
splitIterator = getInputSplits();
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/StatefulSequenceSource.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/StatefulSequenceSource.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/StatefulSequenceSource.java
index 2d74e38..14badf1 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/StatefulSequenceSource.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/functions/source/StatefulSequenceSource.java
@@ -17,24 +17,21 @@
*/
package org.apache.flink.streaming.api.functions.source;
-
-import java.io.IOException;
-
import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.api.common.state.OperatorState;
-import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.checkpoint.Checkpointed;
/**
* A stateful streaming source that emits each number from a given interval exactly once,
* possibly in parallel.
*/
-public class StatefulSequenceSource extends RichParallelSourceFunction<Long> {
+public class StatefulSequenceSource extends RichParallelSourceFunction<Long> implements Checkpointed<Long> {
+
private static final long serialVersionUID = 1L;
private final long start;
private final long end;
- private OperatorState<Long> collected;
+ private long collected;
private volatile boolean isRunning = true;
@@ -62,25 +59,28 @@ public class StatefulSequenceSource extends RichParallelSourceFunction<Long> {
((end - start + 1) % stepSize > (congruence - start)) ?
((end - start + 1) / stepSize + 1) :
((end - start + 1) / stepSize);
-
- Long currentCollected = collected.value();
+
- while (isRunning && currentCollected < toCollect) {
+ while (isRunning && collected < toCollect) {
synchronized (checkpointLock) {
- ctx.collect(currentCollected * stepSize + congruence);
- collected.update(currentCollected + 1);
+ ctx.collect(collected * stepSize + congruence);
+ collected++;
}
- currentCollected = collected.value();
}
}
-
- @Override
- public void open(Configuration conf) throws IOException{
- collected = getRuntimeContext().getOperatorState("collected", 0L, false);
- }
@Override
public void cancel() {
isRunning = false;
}
+
+ @Override
+ public Long snapshotState(long checkpointId, long checkpointTimestamp) {
+ return collected;
+ }
+
+ @Override
+ public void restoreState(Long state) {
+ collected = state;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java
index 2c422d9..55afc93 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java
@@ -27,11 +27,11 @@ import java.util.Map;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.state.StateHandleProvider;
import org.apache.flink.runtime.util.ClassLoaderUtil;
import org.apache.flink.streaming.api.CheckpointingMode;
import org.apache.flink.streaming.api.collector.selector.OutputSelectorWrapper;
import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.state.StateBackend;
import org.apache.flink.streaming.runtime.tasks.StreamTaskException;
import org.apache.flink.util.InstantiationUtil;
@@ -48,9 +48,7 @@ public class StreamConfig implements Serializable {
private static final String CHAINED_OUTPUTS = "chainedOutputs";
private static final String CHAINED_TASK_CONFIG = "chainedTaskConfig_";
private static final String IS_CHAINED_VERTEX = "isChainedSubtask";
- private static final String OUTPUT_NAME = "outputName_";
private static final String VERTEX_NAME = "vertexID";
- private static final String OPERATOR_NAME = "operatorName";
private static final String ITERATION_ID = "iterationId";
private static final String OUTPUT_SELECTOR_WRAPPER = "outputSelectorWrapper";
private static final String SERIALIZEDUDF = "serializedUDF";
@@ -58,8 +56,7 @@ public class StreamConfig implements Serializable {
private static final String BUFFER_TIMEOUT = "bufferTimeout";
private static final String TYPE_SERIALIZER_IN_1 = "typeSerializer_in_1";
private static final String TYPE_SERIALIZER_IN_2 = "typeSerializer_in_2";
- private static final String TYPE_SERIALIZER_OUT_1 = "typeSerializer_out_1";
- private static final String TYPE_SERIALIZER_OUT_2 = "typeSerializer_out_2";
+ private static final String TYPE_SERIALIZER_OUT_1 = "typeSerializer_out";
private static final String ITERATON_WAIT = "iterationWait";
private static final String NONCHAINED_OUTPUTS = "nonChainedOutputs";
private static final String EDGES_IN_ORDER = "edgesInOrder";
@@ -67,10 +64,12 @@ public class StreamConfig implements Serializable {
private static final String IN_STREAM_EDGES = "inStreamEdges";
private static final String CHECKPOINTING_ENABLED = "checkpointing";
- private static final String STATEHANDLE_PROVIDER = "stateHandleProvider";
- private static final String STATE_PARTITIONER = "statePartitioner";
private static final String CHECKPOINT_MODE = "checkpointMode";
+ private static final String STATE_BACKEND = "statebackend";
+ private static final String STATE_PARTITIONER = "statePartitioner";
+ private static final String STATE_KEY_SERIALIZER = "statekeyser";
+
// ------------------------------------------------------------------------
// Default Values
@@ -97,7 +96,6 @@ public class StreamConfig implements Serializable {
// ------------------------------------------------------------------------
// Configured Properties
// ------------------------------------------------------------------------
-
public void setVertexID(Integer vertexID) {
config.setInteger(VERTEX_NAME, vertexID);
@@ -106,15 +104,7 @@ public class StreamConfig implements Serializable {
public Integer getVertexID() {
return config.getInteger(VERTEX_NAME, -1);
}
-
- public void setOperatorName(String name) {
- config.setString(OPERATOR_NAME, name);
- }
-
- public String getOperatorName() {
- return config.getString(OPERATOR_NAME, "Missing");
- }
-
+
public void setTypeSerializerIn1(TypeSerializer<?> serializer) {
setTypeSerializer(TYPE_SERIALIZER_IN_1, serializer);
}
@@ -123,49 +113,29 @@ public class StreamConfig implements Serializable {
setTypeSerializer(TYPE_SERIALIZER_IN_2, serializer);
}
- public void setTypeSerializerOut1(TypeSerializer<?> serializer) {
+ public void setTypeSerializerOut(TypeSerializer<?> serializer) {
setTypeSerializer(TYPE_SERIALIZER_OUT_1, serializer);
}
-
- public void setTypeSerializerOut2(TypeSerializer<?> serializer) {
- setTypeSerializer(TYPE_SERIALIZER_OUT_2, serializer);
- }
-
- @SuppressWarnings("unchecked")
+
public <T> TypeSerializer<T> getTypeSerializerIn1(ClassLoader cl) {
try {
- return (TypeSerializer<T>) InstantiationUtil.readObjectFromConfig(this.config,
- TYPE_SERIALIZER_IN_1, cl);
+ return InstantiationUtil.readObjectFromConfig(this.config, TYPE_SERIALIZER_IN_1, cl);
} catch (Exception e) {
throw new StreamTaskException("Could not instantiate serializer.", e);
}
}
-
- @SuppressWarnings("unchecked")
+
public <T> TypeSerializer<T> getTypeSerializerIn2(ClassLoader cl) {
try {
- return (TypeSerializer<T>) InstantiationUtil.readObjectFromConfig(this.config,
- TYPE_SERIALIZER_IN_2, cl);
- } catch (Exception e) {
- throw new StreamTaskException("Could not instantiate serializer.", e);
- }
- }
-
- @SuppressWarnings("unchecked")
- public <T> TypeSerializer<T> getTypeSerializerOut1(ClassLoader cl) {
- try {
- return (TypeSerializer<T>) InstantiationUtil.readObjectFromConfig(this.config,
- TYPE_SERIALIZER_OUT_1, cl);
+ return InstantiationUtil.readObjectFromConfig(this.config, TYPE_SERIALIZER_IN_2, cl);
} catch (Exception e) {
throw new StreamTaskException("Could not instantiate serializer.", e);
}
}
-
- @SuppressWarnings("unchecked")
- public <T> TypeSerializer<T> getTypeSerializerOut2(ClassLoader cl) {
+
+ public <T> TypeSerializer<T> getTypeSerializerOut(ClassLoader cl) {
try {
- return (TypeSerializer<T>) InstantiationUtil.readObjectFromConfig(this.config,
- TYPE_SERIALIZER_OUT_2, cl);
+ return InstantiationUtil.readObjectFromConfig(this.config, TYPE_SERIALIZER_OUT_1, cl);
} catch (Exception e) {
throw new StreamTaskException("Could not instantiate serializer.", e);
}
@@ -202,9 +172,7 @@ public class StreamConfig implements Serializable {
public <T> T getStreamOperator(ClassLoader cl) {
try {
- @SuppressWarnings("unchecked")
- T result = (T) InstantiationUtil.readObjectFromConfig(this.config, SERIALIZEDUDF, cl);
- return result;
+ return InstantiationUtil.readObjectFromConfig(this.config, SERIALIZEDUDF, cl);
}
catch (ClassNotFoundException e) {
String classLoaderInfo = ClassLoaderUtil.getUserCodeClassLoaderInfo(cl);
@@ -230,12 +198,10 @@ public class StreamConfig implements Serializable {
throw new StreamTaskException("Cannot serialize OutputSelectorWrapper.", e);
}
}
-
- @SuppressWarnings("unchecked")
+
public <T> OutputSelectorWrapper<T> getOutputSelectorWrapper(ClassLoader cl) {
try {
- return (OutputSelectorWrapper<T>) InstantiationUtil.readObjectFromConfig(this.config,
- OUTPUT_SELECTOR_WRAPPER, cl);
+ return InstantiationUtil.readObjectFromConfig(this.config, OUTPUT_SELECTOR_WRAPPER, cl);
} catch (Exception e) {
throw new StreamTaskException("Cannot deserialize and instantiate OutputSelectorWrapper.", e);
}
@@ -280,11 +246,10 @@ public class StreamConfig implements Serializable {
throw new StreamTaskException("Cannot serialize non chained outputs.", e);
}
}
-
- @SuppressWarnings("unchecked")
+
public List<StreamEdge> getNonChainedOutputs(ClassLoader cl) {
try {
- List<StreamEdge> nonChainedOutputs = (List<StreamEdge>) InstantiationUtil.readObjectFromConfig(this.config, NONCHAINED_OUTPUTS, cl);
+ List<StreamEdge> nonChainedOutputs = InstantiationUtil.readObjectFromConfig(this.config, NONCHAINED_OUTPUTS, cl);
return nonChainedOutputs == null ? new ArrayList<StreamEdge>() : nonChainedOutputs;
} catch (Exception e) {
throw new StreamTaskException("Could not instantiate non chained outputs.", e);
@@ -298,11 +263,10 @@ public class StreamConfig implements Serializable {
throw new StreamTaskException("Cannot serialize chained outputs.", e);
}
}
-
- @SuppressWarnings("unchecked")
+
public List<StreamEdge> getChainedOutputs(ClassLoader cl) {
try {
- List<StreamEdge> chainedOutputs = (List<StreamEdge>) InstantiationUtil.readObjectFromConfig(this.config, CHAINED_OUTPUTS, cl);
+ List<StreamEdge> chainedOutputs = InstantiationUtil.readObjectFromConfig(this.config, CHAINED_OUTPUTS, cl);
return chainedOutputs == null ? new ArrayList<StreamEdge>() : chainedOutputs;
} catch (Exception e) {
throw new StreamTaskException("Could not instantiate chained outputs.", e);
@@ -316,12 +280,10 @@ public class StreamConfig implements Serializable {
throw new StreamTaskException("Cannot serialize outward edges.", e);
}
}
-
- @SuppressWarnings("unchecked")
+
public List<StreamEdge> getOutEdges(ClassLoader cl) {
try {
- List<StreamEdge> outEdges = (List<StreamEdge>) InstantiationUtil.readObjectFromConfig(
- this.config, OUT_STREAM_EDGES, cl);
+ List<StreamEdge> outEdges = InstantiationUtil.readObjectFromConfig(this.config, OUT_STREAM_EDGES, cl);
return outEdges == null ? new ArrayList<StreamEdge>() : outEdges;
} catch (Exception e) {
throw new StreamTaskException("Could not instantiate outputs.", e);
@@ -335,12 +297,10 @@ public class StreamConfig implements Serializable {
throw new StreamTaskException("Cannot serialize inward edges.", e);
}
}
-
- @SuppressWarnings("unchecked")
+
public List<StreamEdge> getInPhysicalEdges(ClassLoader cl) {
try {
- List<StreamEdge> inEdges = (List<StreamEdge>) InstantiationUtil.readObjectFromConfig(
- this.config, IN_STREAM_EDGES, cl);
+ List<StreamEdge> inEdges = InstantiationUtil.readObjectFromConfig(this.config, IN_STREAM_EDGES, cl);
return inEdges == null ? new ArrayList<StreamEdge>() : inEdges;
} catch (Exception e) {
throw new StreamTaskException("Could not instantiate inputs.", e);
@@ -378,12 +338,10 @@ public class StreamConfig implements Serializable {
throw new StreamTaskException("Could not serialize outputs in order.", e);
}
}
-
- @SuppressWarnings("unchecked")
+
public List<StreamEdge> getOutEdgesInOrder(ClassLoader cl) {
try {
- List<StreamEdge> outEdgesInOrder = (List<StreamEdge>) InstantiationUtil.readObjectFromConfig(
- this.config, EDGES_IN_ORDER, cl);
+ List<StreamEdge> outEdgesInOrder = InstantiationUtil.readObjectFromConfig(this.config, EDGES_IN_ORDER, cl);
return outEdgesInOrder == null ? new ArrayList<StreamEdge>() : outEdgesInOrder;
} catch (Exception e) {
throw new StreamTaskException("Could not instantiate outputs in order.", e);
@@ -398,31 +356,31 @@ public class StreamConfig implements Serializable {
throw new StreamTaskException("Could not serialize configuration.", e);
}
}
-
- @SuppressWarnings("unchecked")
+
public Map<Integer, StreamConfig> getTransitiveChainedTaskConfigs(ClassLoader cl) {
try {
- Map<Integer, StreamConfig> confs = (Map<Integer, StreamConfig>) InstantiationUtil
- .readObjectFromConfig(this.config, CHAINED_TASK_CONFIG, cl);
+ Map<Integer, StreamConfig> confs = InstantiationUtil.readObjectFromConfig(this.config, CHAINED_TASK_CONFIG, cl);
return confs == null ? new HashMap<Integer, StreamConfig>() : confs;
} catch (Exception e) {
throw new StreamTaskException("Could not instantiate configuration.", e);
}
}
- public void setStateHandleProvider(StateHandleProvider<?> provider) {
+ // ------------------------------------------------------------------------
+ // State backend
+ // ------------------------------------------------------------------------
+
+ public void setStateBackend(StateBackend<?> backend) {
try {
- InstantiationUtil.writeObjectToConfig(provider, this.config, STATEHANDLE_PROVIDER);
- } catch (IOException e) {
+ InstantiationUtil.writeObjectToConfig(backend, this.config, STATE_BACKEND);
+ } catch (Exception e) {
throw new StreamTaskException("Could not serialize stateHandle provider.", e);
}
}
-
- @SuppressWarnings("unchecked")
- public <R> StateHandleProvider<R> getStateHandleProvider(ClassLoader cl) {
+
+ public StateBackend<?> getStateBackend(ClassLoader cl) {
try {
- return (StateHandleProvider<R>) InstantiationUtil
- .readObjectFromConfig(this.config, STATEHANDLE_PROVIDER, cl);
+ return InstantiationUtil.readObjectFromConfig(this.config, STATE_BACKEND, cl);
} catch (Exception e) {
throw new StreamTaskException("Could not instantiate statehandle provider.", e);
}
@@ -435,17 +393,35 @@ public class StreamConfig implements Serializable {
throw new StreamTaskException("Could not serialize state partitioner.", e);
}
}
-
- @SuppressWarnings("unchecked")
+
public KeySelector<?, Serializable> getStatePartitioner(ClassLoader cl) {
try {
- return (KeySelector<?, Serializable>) InstantiationUtil
- .readObjectFromConfig(this.config, STATE_PARTITIONER, cl);
+ return InstantiationUtil.readObjectFromConfig(this.config, STATE_PARTITIONER, cl);
} catch (Exception e) {
throw new StreamTaskException("Could not instantiate state partitioner.", e);
}
}
+
+ public void setStateKeySerializer(TypeSerializer<?> serializer) {
+ try {
+ InstantiationUtil.writeObjectToConfig(serializer, this.config, STATE_KEY_SERIALIZER);
+ } catch (IOException e) {
+ throw new StreamTaskException("Could not serialize state key serializer.", e);
+ }
+ }
+ public <K> TypeSerializer<K> getStateKeySerializer(ClassLoader cl) {
+ try {
+ return InstantiationUtil.readObjectFromConfig(this.config, STATE_KEY_SERIALIZER, cl);
+ } catch (Exception e) {
+ throw new StreamTaskException("Could not instantiate state key serializer from task config.", e);
+ }
+ }
+
+ // ------------------------------------------------------------------------
+ // Miscellansous
+ // ------------------------------------------------------------------------
+
public void setChainStart() {
config.setBoolean(IS_CHAINED_VERTEX, true);
}
@@ -463,7 +439,6 @@ public class StreamConfig implements Serializable {
builder.append("\n=======================");
builder.append("Stream Config");
builder.append("=======================");
- builder.append("\nTask name: ").append(getVertexID());
builder.append("\nNumber of non-chained inputs: ").append(getNumberOfInputs());
builder.append("\nNumber of non-chained outputs: ").append(getNumberOfOutputs());
builder.append("\nOutput names: ").append(getNonChainedOutputs(cl));
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
index 2ca82b1..4c5c19c 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
@@ -43,7 +43,6 @@ import org.apache.flink.api.java.typeutils.MissingTypeInfo;
import org.apache.flink.optimizer.plan.StreamingPlan;
import org.apache.flink.runtime.jobgraph.JobGraph;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
-import org.apache.flink.runtime.state.StateHandleProvider;
import org.apache.flink.streaming.api.CheckpointingMode;
import org.apache.flink.streaming.api.collector.selector.OutputSelector;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
@@ -51,6 +50,7 @@ import org.apache.flink.streaming.api.operators.OutputTypeConfigurable;
import org.apache.flink.streaming.api.operators.StreamOperator;
import org.apache.flink.streaming.api.operators.StreamSource;
import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
+import org.apache.flink.streaming.api.state.StateBackend;
import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner;
import org.apache.flink.streaming.runtime.partitioner.RebalancePartitioner;
import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
@@ -60,9 +60,12 @@ import org.apache.flink.streaming.runtime.tasks.StreamIterationHead;
import org.apache.flink.streaming.runtime.tasks.StreamIterationTail;
import org.apache.flink.streaming.runtime.tasks.TwoInputStreamTask;
import org.apache.sling.commons.json.JSONException;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import static java.util.Objects.requireNonNull;
+
/**
* Class representing the streaming topology. It contains all the information
* necessary to build the jobgraph for the execution.
@@ -93,7 +96,7 @@ public class StreamGraph extends StreamingPlan {
protected Map<Integer, String> vertexIDtoBrokerID;
protected Map<Integer, Long> vertexIDtoLoopTimeout;
- private StateHandleProvider<?> stateHandleProvider;
+ private StateBackend<?> stateBackend;
private Set<Tuple2<StreamNode, StreamNode>> iterationSourceSinkPairs;
private boolean forceCheckpoint = false;
@@ -145,12 +148,12 @@ public class StreamGraph extends StreamingPlan {
this.forceCheckpoint = true;
}
- public void setStateHandleProvider(StateHandleProvider<?> provider) {
- this.stateHandleProvider = provider;
+ public void setStateBackend(StateBackend<?> backend) {
+ this.stateBackend = requireNonNull(backend);
}
- public StateHandleProvider<?> getStateHandleProvider() {
- return this.stateHandleProvider;
+ public StateBackend<?> getStateBackend() {
+ return this.stateBackend;
}
public long getCheckpointingInterval() {
@@ -392,8 +395,10 @@ public class StreamGraph extends StreamingPlan {
}
}
- public void setKey(Integer vertexID, KeySelector<?, ?> key) {
- getStreamNode(vertexID).setStatePartitioner(key);
+ public void setKey(Integer vertexID, KeySelector<?, ?> keySelector, TypeSerializer<?> keySerializer) {
+ StreamNode node = getStreamNode(vertexID);
+ node.setStatePartitioner(keySelector);
+ node.setStateKeySerializer(keySerializer);
}
public void setBufferTimeout(Integer vertexID, long bufferTimeout) {
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
index 774c00b..4a87eb3 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphGenerator.java
@@ -17,8 +17,7 @@
*/
package org.apache.flink.streaming.api.graph;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.source.FileSourceFunction;
@@ -36,8 +35,10 @@ import org.apache.flink.streaming.api.transformations.UnionTransformation;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
+import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -97,17 +98,19 @@ public class StreamGraphGenerator {
private StreamGraphGenerator(StreamExecutionEnvironment env) {
this.streamGraph = new StreamGraph(env);
this.streamGraph.setChaining(env.isChainingEnabled());
+
if (env.getCheckpointInterval() > 0) {
this.streamGraph.setCheckpointingEnabled(true);
this.streamGraph.setCheckpointingInterval(env.getCheckpointInterval());
this.streamGraph.setCheckpointingMode(env.getCheckpointingMode());
}
- this.streamGraph.setStateHandleProvider(env.getStateHandleProvider());
+ this.streamGraph.setStateBackend(env.getStateBackend());
if (env.isForceCheckpointing()) {
this.streamGraph.forceCheckpoint();
}
+
this.env = env;
- this.alreadyTransformed = Maps.newHashMap();
+ this.alreadyTransformed = new HashMap<>();
}
/**
@@ -202,7 +205,7 @@ public class StreamGraphGenerator {
*/
private <T> Collection<Integer> transformUnion(UnionTransformation<T> union) {
List<StreamTransformation<T>> inputs = union.getInputs();
- List<Integer> resultIds = Lists.newArrayList();
+ List<Integer> resultIds = new ArrayList<>();
for (StreamTransformation<T> input: inputs) {
resultIds.addAll(transform(input));
@@ -220,7 +223,7 @@ public class StreamGraphGenerator {
*/
private <T> Collection<Integer> transformPartition(PartitionTransformation<T> partition) {
StreamTransformation<T> input = partition.getInput();
- List<Integer> resultIds = Lists.newArrayList();
+ List<Integer> resultIds = new ArrayList<>();
Collection<Integer> transformedIds = transform(input);
for (Integer transformedId: transformedIds) {
@@ -273,7 +276,7 @@ public class StreamGraphGenerator {
return alreadyTransformed.get(select);
}
- List<Integer> virtualResultIds = Lists.newArrayList();
+ List<Integer> virtualResultIds = new ArrayList<>();
for (int inputId : resultIds) {
int virtualId = StreamTransformation.getNewNodeId();
@@ -301,7 +304,7 @@ public class StreamGraphGenerator {
}
StreamTransformation<T> input = iterate.getInput();
- List<Integer> resultIds = Lists.newArrayList();
+ List<Integer> resultIds = new ArrayList<>();
// first transform the input stream(s) and store the result IDs
resultIds.addAll(transform(input));
@@ -442,7 +445,8 @@ public class StreamGraphGenerator {
if (sink.getStateKeySelector() != null) {
- streamGraph.setKey(sink.getId(), sink.getStateKeySelector());
+ TypeSerializer<?> keySerializer = sink.getStateKeyType().createSerializer(env.getConfig());
+ streamGraph.setKey(sink.getId(), sink.getStateKeySelector(), keySerializer);
}
return Collections.emptyList();
@@ -471,7 +475,11 @@ public class StreamGraphGenerator {
transform.getName());
if (transform.getStateKeySelector() != null) {
- streamGraph.setKey(transform.getId(), transform.getStateKeySelector());
+ TypeSerializer<?> keySerializer = transform.getStateKeyType().createSerializer(env.getConfig());
+ streamGraph.setKey(transform.getId(), transform.getStateKeySelector(), keySerializer);
+ }
+ if (transform.getStateKeyType() != null) {
+
}
streamGraph.setParallelism(transform.getId(), transform.getParallelism());
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java
index 9110cd3..608e648 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java
@@ -50,6 +50,7 @@ public class StreamNode implements Serializable {
private Integer slotSharingID;
private boolean isolatedSlot = false;
private KeySelector<?,?> statePartitioner;
+ private TypeSerializer<?> stateKeySerializer;
private transient StreamOperator<?> operator;
private List<OutputSelector<?>> outputSelectors;
@@ -233,6 +234,14 @@ public class StreamNode implements Serializable {
this.statePartitioner = statePartitioner;
}
+ public TypeSerializer<?> getStateKeySerializer() {
+ return stateKeySerializer;
+ }
+
+ public void setStateKeySerializer(TypeSerializer<?> stateKeySerializer) {
+ this.stateKeySerializer = stateKeySerializer;
+ }
+
@Override
public boolean equals(Object o) {
if (this == o) {
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java
index d8e81cf..45cfff1 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java
@@ -43,7 +43,7 @@ import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup;
import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
import org.apache.flink.streaming.api.CheckpointingMode;
import org.apache.flink.streaming.api.operators.StreamOperator;
-import org.apache.flink.streaming.api.operators.StreamOperator.ChainingStrategy;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner;
import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
import org.apache.flink.streaming.runtime.tasks.StreamIterationHead;
@@ -211,13 +211,9 @@ public class StreamingJobGraphGenerator {
for (StreamEdge chainable : chainedOutputs) {
outputChainedNames.add(chainedNames.get(chainable.getTargetId()));
}
- String returnOperatorName = operatorName + " -> ("
- + StringUtils.join(outputChainedNames, ", ") + ")";
- return returnOperatorName;
+ return operatorName + " -> (" + StringUtils.join(outputChainedNames, ", ") + ")";
} else if (chainedOutputs.size() == 1) {
- String returnOperatorName = operatorName + " -> "
- + chainedNames.get(chainedOutputs.get(0).getTargetId());
- return returnOperatorName;
+ return operatorName + " -> " + chainedNames.get(chainedOutputs.get(0).getTargetId());
} else {
return operatorName;
}
@@ -249,9 +245,7 @@ public class StreamingJobGraphGenerator {
builtVertices.add(vertexID);
jobGraph.addVertex(jobVertex);
- StreamConfig retConfig = new StreamConfig(jobVertex.getConfiguration());
- retConfig.setOperatorName(chainedNames.get(vertexID));
- return retConfig;
+ return new StreamConfig(jobVertex.getConfiguration());
}
@SuppressWarnings("unchecked")
@@ -265,7 +259,7 @@ public class StreamingJobGraphGenerator {
config.setTypeSerializerIn1(vertex.getTypeSerializerIn1());
config.setTypeSerializerIn2(vertex.getTypeSerializerIn2());
- config.setTypeSerializerOut1(vertex.getTypeSerializerOut());
+ config.setTypeSerializerOut(vertex.getTypeSerializerOut());
config.setStreamOperator(vertex.getOperator());
config.setOutputSelectorWrapper(vertex.getOutputSelectorWrapper());
@@ -277,13 +271,14 @@ public class StreamingJobGraphGenerator {
config.setCheckpointingEnabled(streamGraph.isCheckpointingEnabled());
if (streamGraph.isCheckpointingEnabled()) {
config.setCheckpointMode(streamGraph.getCheckpointingMode());
- config.setStateHandleProvider(streamGraph.getStateHandleProvider());
+ config.setStateBackend(streamGraph.getStateBackend());
} else {
// the at least once input handler is slightly cheaper (in the absence of checkpoints),
// so we use that one if checkpointing is not enabled
config.setCheckpointMode(CheckpointingMode.AT_LEAST_ONCE);
}
config.setStatePartitioner((KeySelector<?, Serializable>) vertex.getStatePartitioner());
+ config.setStateKeySerializer(vertex.getStateKeySerializer());
Class<? extends AbstractInvokable> vertexClass = vertex.getJobVertexClass();
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
index 87041eb..e99d54d 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
@@ -19,61 +19,302 @@
package org.apache.flink.streaming.api.operators;
import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.configuration.Configuration;
+import org.apache.flink.api.common.state.OperatorState;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.state.KvState;
+import org.apache.flink.streaming.api.state.KvStateSnapshot;
+import org.apache.flink.streaming.api.state.StateBackend;
+import org.apache.flink.streaming.runtime.operators.Triggerable;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.apache.flink.streaming.runtime.tasks.StreamTaskState;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
/**
- * Base class for all stream operators.
- *
- * Operators that contain a user function should extend the class
- * {@link AbstractUdfStreamOperator} instead (which is a specialized subclass of this class).
+ * Base class for all stream operators. Operators that contain a user function should extend the class
+ * {@link AbstractUdfStreamOperator} instead (which is a specialized subclass of this class).
*
+ * <p>For concrete implementations, one of the following two interfaces must also be implemented, to
+ * mark the operator as unary or binary:
+ * {@link org.apache.flink.streaming.api.operators.OneInputStreamOperator} or
+ * {@link org.apache.flink.streaming.api.operators.TwoInputStreamOperator}.
+ *
+ * <p>Methods of {@code StreamOperator} are guaranteed not to be called concurrently. Also, if using
+ * the timer service, timer callbacks are also guaranteed not to be called concurrently with
+ * methods on {@code StreamOperator}.
+ *
* @param <OUT> The output type of the operator
*/
-public abstract class AbstractStreamOperator<OUT> implements StreamOperator<OUT> {
+public abstract class AbstractStreamOperator<OUT>
+ implements StreamOperator<OUT>, java.io.Serializable {
private static final long serialVersionUID = 1L;
+
+ /** The logger used by the operator class and its subclasses */
+ protected static final Logger LOG = LoggerFactory.getLogger(AbstractStreamOperator.class);
- protected transient StreamingRuntimeContext runtimeContext;
+ // ----------- configuration properties -------------
- protected transient ExecutionConfig executionConfig;
+ // A sane default for most operators
+ protected ChainingStrategy chainingStrategy = ChainingStrategy.HEAD;
+
+ private boolean inputCopyDisabled = false;
+
+ // ---------------- runtime fields ------------------
- protected transient Output<StreamRecord<OUT>> output;
+ /** The task that contains this operator (and other operators in the same chain) */
+ private transient StreamTask<?, ?> container;
+
+ private transient StreamConfig config;
- protected boolean inputCopyDisabled = false;
+ protected transient Output<StreamRecord<OUT>> output;
- // A sane default for most operators
- protected ChainingStrategy chainingStrategy = ChainingStrategy.HEAD;
+ /** The runtime context for UDFs */
+ private transient StreamingRuntimeContext runtimeContext;
+ // ---------------- key/value state ------------------
+
+ /** key selector used to get the key for the state. Non-null only is the operator uses key/value state */
+ private transient KeySelector<?, ?> stateKeySelector;
+
+ private transient KvState<?, ?, ?> keyValueState;
+
+ private transient KvStateSnapshot<?, ?, ?> keyValueStateSnapshot;
+
// ------------------------------------------------------------------------
// Life Cycle
// ------------------------------------------------------------------------
-
+
@Override
- public void setup(Output<StreamRecord<OUT>> output, StreamingRuntimeContext runtimeContext) {
+ public void setup(StreamTask<?, ?> containingTask, StreamConfig config, Output<StreamRecord<OUT>> output) {
+ this.container = containingTask;
+ this.config = config;
this.output = output;
- this.executionConfig = runtimeContext.getExecutionConfig();
- this.runtimeContext = runtimeContext;
+ this.runtimeContext = new StreamingRuntimeContext(this, container.getEnvironment(), container.getAccumulatorMap());
}
/**
- * This default implementation of the interface method does nothing.
+ * This method is called immediately before any elements are processed, it should contain the
+ * operator's initialization logic.
+ *
+ * <p>The default implementation does nothing.
+ *
+ * @throws Exception An exception in this method causes the operator to fail.
*/
@Override
- public void open(Configuration parameters) throws Exception {}
+ public void open() throws Exception {}
/**
- * This default implementation of the interface method does nothing.
+ * This method is called after all records have been added to the operators via the methods
+ * {@link org.apache.flink.streaming.api.operators.OneInputStreamOperator#processElement(StreamRecord)}, or
+ * {@link org.apache.flink.streaming.api.operators.TwoInputStreamOperator#processElement1(StreamRecord)} and
+ * {@link org.apache.flink.streaming.api.operators.TwoInputStreamOperator#processElement2(StreamRecord)}.
+
+ * <p>The method is expected to flush all remaining buffered data. Exceptions during this flushing
+ * of buffered should be propagated, in order to cause the operation to be recognized asa failed,
+ * because the last data items are not processed properly.
+ *
+ * @throws Exception An exception in this method causes the operator to fail.
*/
@Override
public void close() throws Exception {}
+
+ /**
+ * This method is called at the very end of the operator's life, both in the case of a successful
+ * completion of the operation, and in the case of a failure and canceling.
+ *
+ * This method is expected to make a thorough effort to release all resources
+ * that the operator has acquired.
+ */
+ @Override
+ public void dispose() {
+ if (keyValueState != null) {
+ keyValueState.dispose();
+ }
+ }
+
+ // ------------------------------------------------------------------------
+ // Checkpointing
+ // ------------------------------------------------------------------------
+
+ @Override
+ public StreamTaskState snapshotOperatorState(long checkpointId, long timestamp) throws Exception {
+ // here, we deal with operator checkpoints and key/value state snapshots
+
+ StreamTaskState state = new StreamTaskState();
+
+ // (1) checkpoint the operator, if the operator is stateful
+
+ // (2) draw a snapshot of the key/value state
+ if (keyValueState != null) {
+ KvStateSnapshot<?, ?, ?> snapshot = keyValueState.shapshot(checkpointId, timestamp);
+ state.setKvState(snapshot);
+ }
+
+ return state;
+ }
+
+ @Override
+ public void restoreState(StreamTaskState state) throws Exception {
+ // (1) checkpoint the operator, if the operator is stateful
+
+ // (2) restore the key/value state. the actual restore happens lazily, when the function requests
+ // the state again, because the restore method needs information provided by the user function
+ keyValueStateSnapshot = state.getKvState();
+ }
+
+ @Override
+ public void notifyOfCompletedCheckpoint(long checkpointId) throws Exception {
+ // by default, nothing needs a notification of checkpoint completion
+ }
+
+ // ------------------------------------------------------------------------
+ // Properties and Services
+ // ------------------------------------------------------------------------
+
+ /**
+ * Gets the execution config defined on the execution environment of the job to which this
+ * operator belongs.
+ *
+ * @return The job's execution config.
+ */
+ public ExecutionConfig getExecutionConfig() {
+ return container.getExecutionConfig();
+ }
+
+ public StreamConfig getOperatorConfig() {
+ return config;
+ }
+
+ public StreamTask<?, ?> getContainingTask() {
+ return container;
+ }
+
+ public ClassLoader getUserCodeClassloader() {
+ return container.getUserCodeClassLoader();
+ }
+
+ /**
+ * Returns a context that allows the operator to query information about the execution and also
+ * to interact with systems such as broadcast variables and managed state. This also allows
+ * to register timers.
+ */
+ public StreamingRuntimeContext getRuntimeContext() {
+ return runtimeContext;
+ }
+
+ public StateBackend<?> getStateBackend() {
+ return container.getStateBackend();
+ }
/**
- * This default implementation of the interface method does nothing.
+ * Register a timer callback. At the specified time the {@link Triggerable} will be invoked.
+ * This call is guaranteed to not happen concurrently with method calls on the operator.
+ *
+ * @param time The absolute time in milliseconds.
+ * @param target The target to be triggered.
*/
+ protected void registerTimer(long time, Triggerable target) {
+ container.registerTimer(time, target);
+ }
+
+ /**
+ * Creates a key/value state handle, using the state backend configured for this task.
+ *
+ * @param stateType The type information for the state type, used for managed memory and state snapshots.
+ * @param defaultValue The default value that the state should return for keys that currently have
+ * no value associated with them
+ *
+ * @param <V> The type of the state value.
+ *
+ * @return The key/value state for this operator.
+ *
+ * @throws IllegalStateException Thrown, if the key/value state was already initialized.
+ * @throws Exception Thrown, if the state backend cannot create the key/value state.
+ */
+ protected <V> OperatorState<V> createKeyValueState(
+ TypeInformation<V> stateType, V defaultValue) throws Exception
+ {
+ return createKeyValueState(stateType.createSerializer(getExecutionConfig()), defaultValue);
+ }
+
+ /**
+ * Creates a key/value state handle, using the state backend configured for this task.
+ *
+ * @param valueSerializer The type serializer for the state type, used for managed memory and state snapshots.
+ * @param defaultValue The default value that the state should return for keys that currently have
+ * no value associated with them
+ *
+ * @param <K> The type of the state key.
+ * @param <V> The type of the state value.
+ * @param <Backend> The type of the state backend that creates the key/value state.
+ *
+ * @return The key/value state for this operator.
+ *
+ * @throws IllegalStateException Thrown, if the key/value state was already initialized.
+ * @throws Exception Thrown, if the state backend cannot create the key/value state.
+ */
+ protected <K, V, Backend extends StateBackend<Backend>> OperatorState<V> createKeyValueState(
+ TypeSerializer<V> valueSerializer, V defaultValue) throws Exception
+ {
+ if (keyValueState != null) {
+ throw new IllegalStateException("The key/value state has already been created");
+ }
+
+ // first time state access, make sure we load the state partitioner
+ if (stateKeySelector == null) {
+ stateKeySelector = config.getStatePartitioner(getUserCodeClassloader());
+ if (stateKeySelector == null) {
+ throw new UnsupportedOperationException("The function or operator is not executed " +
+ "on a KeyedStream and can hence not access the key/value state");
+ }
+ }
+
+ // create the key and value serializers
+ TypeSerializer<K> keySerializer = config.getStateKeySerializer(getUserCodeClassloader());
+ if (keySerializer == null) {
+ throw new Exception("State key serializer has not been configured in the config.");
+ }
+
+ @SuppressWarnings("unchecked")
+ Backend stateBackend = (Backend) container.getStateBackend();
+
+ // check whether we restore the key/value state from a snapshot, or create a new blank one
+ if (keyValueStateSnapshot != null) {
+ @SuppressWarnings("unchecked")
+ KvStateSnapshot<K, V, Backend> snapshot = (KvStateSnapshot<K, V, Backend>) keyValueStateSnapshot;
+
+ KvState<K, V, Backend> kvstate = snapshot.restoreState(
+ stateBackend, keySerializer, valueSerializer, defaultValue, getUserCodeClassloader());
+ keyValueState = kvstate;
+
+ // make sure we have no redundant copies in memory, let the GC clean up
+ keyValueStateSnapshot = null;
+
+ return kvstate;
+ }
+ else {
+ // create a new blank key/value state
+ KvState<K, V, Backend> kvstate = stateBackend.createKvState(keySerializer, valueSerializer, defaultValue);
+ keyValueState = kvstate;
+ return kvstate;
+ }
+ }
+
@Override
- public void dispose() {}
+ @SuppressWarnings({"unchecked", "rawtypes"})
+ public void setKeyContextElement(StreamRecord record) throws Exception {
+ if (stateKeySelector != null && keyValueState != null) {
+ KvState kv = keyValueState;
+ KeySelector selector = stateKeySelector;
+ kv.setCurrentKey(selector.getKey(record.getValue()));
+ }
+ }
// ------------------------------------------------------------------------
// Context and chaining properties
@@ -83,12 +324,12 @@ public abstract class AbstractStreamOperator<OUT> implements StreamOperator<OUT>
public final void setChainingStrategy(ChainingStrategy strategy) {
this.chainingStrategy = strategy;
}
-
+
@Override
public final ChainingStrategy getChainingStrategy() {
return chainingStrategy;
}
-
+
@Override
public boolean isInputCopyingDisabled() {
return inputCopyDisabled;
@@ -96,14 +337,9 @@ public abstract class AbstractStreamOperator<OUT> implements StreamOperator<OUT>
/**
* Enable object-reuse for this operator instance. This overrides the setting in
- * the {@link org.apache.flink.api.common.ExecutionConfig}/
+ * the {@link org.apache.flink.api.common.ExecutionConfig}
*/
public void disableInputCopy() {
this.inputCopyDisabled = true;
}
-
- @Override
- public StreamingRuntimeContext getRuntimeContext(){
- return runtimeContext;
- }
}
[04/24] flink git commit: [FLINK-2808] [streaming] Refactor and
extend state backend abstraction
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
index 89672df..671544e 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
@@ -18,15 +18,18 @@
package org.apache.flink.streaming.runtime.operators.windowing;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.accumulators.Accumulator;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.execution.Environment;
import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
+import org.apache.flink.streaming.api.graph.StreamConfig;
import org.apache.flink.streaming.api.operators.Output;
import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
import org.apache.flink.streaming.runtime.operators.Triggerable;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.tasks.StreamTask;
-import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext;
import org.apache.flink.util.Collector;
import org.junit.After;
@@ -36,6 +39,7 @@ import org.mockito.stubbing.Answer;
import java.util.Arrays;
import java.util.Collections;
+import java.util.HashMap;
import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.Executors;
@@ -152,36 +156,34 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
try {
@SuppressWarnings("unchecked")
final Output<StreamRecord<String>> mockOut = mock(Output.class);
-
- final StreamingRuntimeContext mockContext = mock(StreamingRuntimeContext.class);
- when(mockContext.getTaskName()).thenReturn("Test task name");
+ final StreamTask<?, ?> mockTask = createMockTask();
AbstractAlignedProcessingTimeWindowOperator<String, String, String, ?> op;
op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector, 5000, 1000);
- op.setup(mockOut, mockContext);
- op.open(new Configuration());
+ op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
+ op.open();
assertTrue(op.getNextSlideTime() % 1000 == 0);
assertTrue(op.getNextEvaluationTime() % 1000 == 0);
op.dispose();
op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector, 1000, 1000);
- op.setup(mockOut, mockContext);
- op.open(new Configuration());
+ op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
+ op.open();
assertTrue(op.getNextSlideTime() % 1000 == 0);
assertTrue(op.getNextEvaluationTime() % 1000 == 0);
op.dispose();
op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector, 1500, 1000);
- op.setup(mockOut, mockContext);
- op.open(new Configuration());
+ op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
+ op.open();
assertTrue(op.getNextSlideTime() % 500 == 0);
assertTrue(op.getNextEvaluationTime() % 1000 == 0);
op.dispose();
op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector, 1200, 1100);
- op.setup(mockOut, mockContext);
- op.open(new Configuration());
+ op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
+ op.open();
assertTrue(op.getNextSlideTime() % 100 == 0);
assertTrue(op.getNextEvaluationTime() % 1100 == 0);
op.dispose();
@@ -194,25 +196,27 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
@Test
public void testTumblingWindow() {
+ final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
try {
final int windowSize = 50;
final CollectingOutput<Integer> out = new CollectingOutput<>(windowSize);
-
- final StreamingRuntimeContext mockContext = mock(StreamingRuntimeContext.class);
- when(mockContext.getTaskName()).thenReturn("Test task name");
+ final Object lock = new Object();
+ final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
// tumbling window that triggers every 20 milliseconds
AbstractAlignedProcessingTimeWindowOperator<Integer, Integer, Integer, ?> op =
new AccumulatingProcessingTimeWindowOperator<>(
validatingIdentityFunction, identitySelector, windowSize, windowSize);
- op.setup(out, mockContext);
- op.open(new Configuration());
+ op.setup(mockTask, new StreamConfig(new Configuration()), out);
+ op.open();
final int numElements = 1000;
for (int i = 0; i < numElements; i++) {
- op.processElement(new StreamRecord<Integer>(i));
+ synchronized (lock) {
+ op.processElement(new StreamRecord<Integer>(i));
+ }
Thread.sleep(1);
}
@@ -232,27 +236,32 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
e.printStackTrace();
fail(e.getMessage());
}
+ finally {
+ timerService.shutdown();
+ }
}
@Test
public void testSlidingWindow() {
+ final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
try {
final CollectingOutput<Integer> out = new CollectingOutput<>(50);
-
- final StreamingRuntimeContext mockContext = mock(StreamingRuntimeContext.class);
- when(mockContext.getTaskName()).thenReturn("Test task name");
-
+ final Object lock = new Object();
+ final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
+
// tumbling window that triggers every 20 milliseconds
AbstractAlignedProcessingTimeWindowOperator<Integer, Integer, Integer, ?> op =
new AccumulatingProcessingTimeWindowOperator<>(validatingIdentityFunction, identitySelector, 150, 50);
- op.setup(out, mockContext);
- op.open(new Configuration());
+ op.setup(mockTask, new StreamConfig(new Configuration()), out);
+ op.open();
final int numElements = 1000;
for (int i = 0; i < numElements; i++) {
- op.processElement(new StreamRecord<Integer>(i));
+ synchronized (lock) {
+ op.processElement(new StreamRecord<Integer>(i));
+ }
Thread.sleep(1);
}
@@ -288,6 +297,9 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
e.printStackTrace();
fail(e.getMessage());
}
+ finally {
+ timerService.shutdown();
+ }
}
@Test
@@ -296,39 +308,15 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
try {
final CollectingOutput<Integer> out = new CollectingOutput<>(50);
-
- final StreamingRuntimeContext mockContext = mock(StreamingRuntimeContext.class);
- when(mockContext.getTaskName()).thenReturn("Test task name");
-
final Object lock = new Object();
-
- doAnswer(new Answer<Void>() {
- @Override
- public Void answer(InvocationOnMock invocationOnMock) throws Throwable {
- final Long timestamp = (Long) invocationOnMock.getArguments()[0];
- final Triggerable target = (Triggerable) invocationOnMock.getArguments()[1];
- timerService.schedule(
- new Callable<Object>() {
- @Override
- public Object call() throws Exception {
- synchronized (lock) {
- target.trigger(timestamp);
- }
- return null;
- }
- },
- timestamp - System.currentTimeMillis(),
- TimeUnit.MILLISECONDS);
- return null;
- }
- }).when(mockContext).registerTimer(anyLong(), any(Triggerable.class));
+ final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
// tumbling window that triggers every 20 milliseconds
AbstractAlignedProcessingTimeWindowOperator<Integer, Integer, Integer, ?> op =
new AccumulatingProcessingTimeWindowOperator<>(validatingIdentityFunction, identitySelector, 50, 50);
- op.setup(out, mockContext);
- op.open(new Configuration());
+ op.setup(mockTask, new StreamConfig(new Configuration()), out);
+ op.open();
synchronized (lock) {
op.processElement(new StreamRecord<Integer>(1));
@@ -360,7 +348,8 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
catch (Exception e) {
e.printStackTrace();
fail(e.getMessage());
- } finally {
+ }
+ finally {
timerService.shutdown();
}
}
@@ -371,39 +360,15 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
try {
final CollectingOutput<Integer> out = new CollectingOutput<>(50);
-
- final StreamingRuntimeContext mockContext = mock(StreamingRuntimeContext.class);
- when(mockContext.getTaskName()).thenReturn("Test task name");
-
final Object lock = new Object();
-
- doAnswer(new Answer<Void>() {
- @Override
- public Void answer(InvocationOnMock invocationOnMock) throws Throwable {
- final Long timestamp = (Long) invocationOnMock.getArguments()[0];
- final Triggerable target = (Triggerable) invocationOnMock.getArguments()[1];
- timerService.schedule(
- new Callable<Object>() {
- @Override
- public Object call() throws Exception {
- synchronized (lock) {
- target.trigger(timestamp);
- }
- return null;
- }
- },
- timestamp - System.currentTimeMillis(),
- TimeUnit.MILLISECONDS);
- return null;
- }
- }).when(mockContext).registerTimer(anyLong(), any(Triggerable.class));
+ final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
// tumbling window that triggers every 20 milliseconds
AbstractAlignedProcessingTimeWindowOperator<Integer, Integer, Integer, ?> op =
new AccumulatingProcessingTimeWindowOperator<>(validatingIdentityFunction, identitySelector, 150, 50);
- op.setup(out, mockContext);
- op.open(new Configuration());
+ op.setup(mockTask, new StreamConfig(new Configuration()), out);
+ op.open();
synchronized (lock) {
op.processElement(new StreamRecord<Integer>(1));
@@ -426,31 +391,34 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
catch (Exception e) {
e.printStackTrace();
fail(e.getMessage());
- } finally {
+ }
+ finally {
timerService.shutdown();
}
}
@Test
public void testEmitTrailingDataOnClose() {
+ final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
try {
final CollectingOutput<Integer> out = new CollectingOutput<>();
-
- final StreamingRuntimeContext mockContext = mock(StreamingRuntimeContext.class);
- when(mockContext.getTaskName()).thenReturn("Test task name");
+ final Object lock = new Object();
+ final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
// the operator has a window time that is so long that it will not fire in this test
final long oneYear = 365L * 24 * 60 * 60 * 1000;
AbstractAlignedProcessingTimeWindowOperator<Integer, Integer, Integer, ?> op =
new AccumulatingProcessingTimeWindowOperator<>(validatingIdentityFunction, identitySelector,
oneYear, oneYear);
-
- op.setup(out, mockContext);
- op.open(new Configuration());
+
+ op.setup(mockTask, new StreamConfig(new Configuration()), out);
+ op.open();
List<Integer> data = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10);
for (Integer i : data) {
- op.processElement(new StreamRecord<Integer>(i));
+ synchronized (lock) {
+ op.processElement(new StreamRecord<Integer>(i));
+ }
}
op.close();
@@ -465,15 +433,18 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
e.printStackTrace();
fail(e.getMessage());
}
+ finally {
+ timerService.shutdown();
+ }
}
@Test
public void testPropagateExceptionsFromClose() {
+ final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
try {
final CollectingOutput<Integer> out = new CollectingOutput<>();
-
- final StreamingRuntimeContext mockContext = mock(StreamingRuntimeContext.class);
- when(mockContext.getTaskName()).thenReturn("Test task name");
+ final Object lock = new Object();
+ final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
WindowFunction<Integer, Integer, Integer, TimeWindow> failingFunction = new FailingFunction(100);
@@ -483,11 +454,13 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
new AccumulatingProcessingTimeWindowOperator<>(
failingFunction, identitySelector, hundredYears, hundredYears);
- op.setup(out, mockContext);
- op.open(new Configuration());
+ op.setup(mockTask, new StreamConfig(new Configuration()), out);
+ op.open();
for (int i = 0; i < 150; i++) {
- op.processElement(new StreamRecord<Integer>(i));
+ synchronized (lock) {
+ op.processElement(new StreamRecord<Integer>(i));
+ }
}
try {
@@ -506,6 +479,9 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
e.printStackTrace();
fail(e.getMessage());
}
+ finally {
+ timerService.shutdown();
+ }
}
// ------------------------------------------------------------------------
@@ -551,4 +527,49 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
}
}
}
+
+ private static StreamTask<?, ?> createMockTask() {
+ StreamTask<?, ?> task = mock(StreamTask.class);
+ when(task.getAccumulatorMap()).thenReturn(new HashMap<String, Accumulator<?, ?>>());
+ when(task.getName()).thenReturn("Test task name");
+ when(task.getExecutionConfig()).thenReturn(new ExecutionConfig());
+
+ Environment env = mock(Environment.class);
+ when(env.getIndexInSubtaskGroup()).thenReturn(0);
+ when(env.getNumberOfSubtasks()).thenReturn(1);
+ when(env.getUserClassLoader()).thenReturn(AggregatingAlignedProcessingTimeWindowOperatorTest.class.getClassLoader());
+
+ when(task.getEnvironment()).thenReturn(env);
+
+ return task;
+ }
+
+ private static StreamTask<?, ?> createMockTaskWithTimer(
+ final ScheduledExecutorService timerService, final Object lock)
+ {
+ StreamTask<?, ?> mockTask = createMockTask();
+
+ doAnswer(new Answer<Void>() {
+ @Override
+ public Void answer(InvocationOnMock invocationOnMock) throws Throwable {
+ final Long timestamp = (Long) invocationOnMock.getArguments()[0];
+ final Triggerable target = (Triggerable) invocationOnMock.getArguments()[1];
+ timerService.schedule(
+ new Callable<Object>() {
+ @Override
+ public Object call() throws Exception {
+ synchronized (lock) {
+ target.trigger(timestamp);
+ }
+ return null;
+ }
+ },
+ timestamp - System.currentTimeMillis(),
+ TimeUnit.MILLISECONDS);
+ return null;
+ }
+ }).when(mockTask).registerTimer(anyLong(), any(Triggerable.class));
+
+ return mockTask;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java
index fa90e4a..106e833 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java
@@ -18,14 +18,17 @@
package org.apache.flink.streaming.runtime.operators.windowing;
+import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.accumulators.Accumulator;
import org.apache.flink.api.common.functions.ReduceFunction;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.execution.Environment;
+import org.apache.flink.streaming.api.graph.StreamConfig;
import org.apache.flink.streaming.api.operators.Output;
import org.apache.flink.streaming.runtime.operators.Triggerable;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.tasks.StreamTask;
-import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext;
import org.junit.After;
import org.junit.Test;
@@ -34,6 +37,7 @@ import org.mockito.stubbing.Answer;
import java.util.Arrays;
import java.util.Collections;
+import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.concurrent.Callable;
@@ -149,36 +153,34 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
try {
@SuppressWarnings("unchecked")
final Output<StreamRecord<String>> mockOut = mock(Output.class);
+ final StreamTask<?, ?> mockTask = createMockTask();
- final StreamingRuntimeContext mockContext = mock(StreamingRuntimeContext.class);
- when(mockContext.getTaskName()).thenReturn("Test task name");
-
AggregatingProcessingTimeWindowOperator<String, String> op;
op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector, 5000, 1000);
- op.setup(mockOut, mockContext);
- op.open(new Configuration());
+ op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
+ op.open();
assertTrue(op.getNextSlideTime() % 1000 == 0);
assertTrue(op.getNextEvaluationTime() % 1000 == 0);
op.dispose();
op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector, 1000, 1000);
- op.setup(mockOut, mockContext);
- op.open(new Configuration());
+ op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
+ op.open();
assertTrue(op.getNextSlideTime() % 1000 == 0);
assertTrue(op.getNextEvaluationTime() % 1000 == 0);
op.dispose();
op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector, 1500, 1000);
- op.setup(mockOut, mockContext);
- op.open(new Configuration());
+ op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
+ op.open();
assertTrue(op.getNextSlideTime() % 500 == 0);
assertTrue(op.getNextEvaluationTime() % 1000 == 0);
op.dispose();
op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector, 1200, 1100);
- op.setup(mockOut, mockContext);
- op.open(new Configuration());
+ op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
+ op.open();
assertTrue(op.getNextSlideTime() % 100 == 0);
assertTrue(op.getNextEvaluationTime() % 1100 == 0);
op.dispose();
@@ -191,19 +193,20 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
@Test
public void testTumblingWindowUniqueElements() {
+ final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
try {
final int windowSize = 50;
final CollectingOutput<Integer> out = new CollectingOutput<>(windowSize);
-
- final StreamingRuntimeContext mockContext = mock(StreamingRuntimeContext.class);
- when(mockContext.getTaskName()).thenReturn("Test task name");
AggregatingProcessingTimeWindowOperator<Integer, Integer> op =
new AggregatingProcessingTimeWindowOperator<>(
sumFunction, identitySelector, windowSize, windowSize);
-
- op.setup(out, mockContext);
- op.open(new Configuration());
+
+ final Object lock = new Object();
+ final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
+
+ op.setup(mockTask, new StreamConfig(new Configuration()), out);
+ op.open();
final int numElements = 1000;
@@ -228,6 +231,9 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
e.printStackTrace();
fail(e.getMessage());
}
+ finally {
+ timerService.shutdownNow();
+ }
}
@Test
@@ -239,37 +245,15 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
final int windowSize = 50;
final CollectingOutput<Integer> out = new CollectingOutput<>(windowSize);
- final StreamingRuntimeContext mockContext = mock(StreamingRuntimeContext.class);
- when(mockContext.getTaskName()).thenReturn("Test task name");
-
final Object lock = new Object();
- doAnswer(new Answer<Void>() {
- @Override
- public Void answer(InvocationOnMock invocationOnMock) throws Throwable {
- final Long timestamp = (Long) invocationOnMock.getArguments()[0];
- final Triggerable target = (Triggerable) invocationOnMock.getArguments()[1];
- timerService.schedule(
- new Callable<Object>() {
- @Override
- public Object call() throws Exception {
- synchronized (lock) {
- target.trigger(timestamp);
- }
- return null;
- }
- },
- timestamp - System.currentTimeMillis(),
- TimeUnit.MILLISECONDS);
- return null;
- }
- }).when(mockContext).registerTimer(anyLong(), any(Triggerable.class));
+ final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
AggregatingProcessingTimeWindowOperator<Integer, Integer> op =
new AggregatingProcessingTimeWindowOperator<>(
sumFunction, identitySelector, windowSize, windowSize);
-
- op.setup(out, mockContext);
- op.open(new Configuration());
+
+ op.setup(mockTask, new StreamConfig(new Configuration()), out);
+ op.open();
final int numWindows = 10;
@@ -315,23 +299,26 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
@Test
public void testSlidingWindow() {
+ final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
try {
final CollectingOutput<Integer> out = new CollectingOutput<>(50);
- final StreamingRuntimeContext mockContext = mock(StreamingRuntimeContext.class);
- when(mockContext.getTaskName()).thenReturn("Test task name");
+ final Object lock = new Object();
+ final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
// tumbling window that triggers every 20 milliseconds
AggregatingProcessingTimeWindowOperator<Integer, Integer> op =
new AggregatingProcessingTimeWindowOperator<>(sumFunction, identitySelector, 150, 50);
- op.setup(out, mockContext);
- op.open(new Configuration());
+ op.setup(mockTask, new StreamConfig(new Configuration()), out);
+ op.open();
final int numElements = 1000;
for (int i = 0; i < numElements; i++) {
- op.processElement(new StreamRecord<Integer>(i));
+ synchronized (lock) {
+ op.processElement(new StreamRecord<Integer>(i));
+ }
Thread.sleep(1);
}
@@ -366,6 +353,9 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
e.printStackTrace();
fail(e.getMessage());
}
+ finally {
+ timerService.shutdownNow();
+ }
}
@Test
@@ -374,38 +364,15 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
try {
final CollectingOutput<Integer> out = new CollectingOutput<>(50);
-
- final StreamingRuntimeContext mockContext = mock(StreamingRuntimeContext.class);
- when(mockContext.getTaskName()).thenReturn("Test task name");
-
final Object lock = new Object();
- doAnswer(new Answer<Void>() {
- @Override
- public Void answer(InvocationOnMock invocationOnMock) throws Throwable {
- final Long timestamp = (Long) invocationOnMock.getArguments()[0];
- final Triggerable target = (Triggerable) invocationOnMock.getArguments()[1];
- timerService.schedule(
- new Callable<Object>() {
- @Override
- public Object call() throws Exception {
- synchronized (lock) {
- target.trigger(timestamp);
- }
- return null;
- }
- },
- timestamp - System.currentTimeMillis(),
- TimeUnit.MILLISECONDS);
- return null;
- }
- }).when(mockContext).registerTimer(anyLong(), any(Triggerable.class));
+ final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
// tumbling window that triggers every 20 milliseconds
AggregatingProcessingTimeWindowOperator<Integer, Integer> op =
new AggregatingProcessingTimeWindowOperator<>(sumFunction, identitySelector, 150, 50);
- op.setup(out, mockContext);
- op.open(new Configuration());
+ op.setup(mockTask, new StreamConfig(new Configuration()), out);
+ op.open();
synchronized (lock) {
op.processElement(new StreamRecord<Integer>(1));
@@ -428,30 +395,33 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
catch (Exception e) {
e.printStackTrace();
fail(e.getMessage());
- } finally {
+ }
+ finally {
timerService.shutdown();
}
}
@Test
public void testEmitTrailingDataOnClose() {
+ final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
try {
final CollectingOutput<Integer> out = new CollectingOutput<>();
-
- final StreamingRuntimeContext mockContext = mock(StreamingRuntimeContext.class);
- when(mockContext.getTaskName()).thenReturn("Test task name");
+ final Object lock = new Object();
+ final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
// the operator has a window time that is so long that it will not fire in this test
final long oneYear = 365L * 24 * 60 * 60 * 1000;
AggregatingProcessingTimeWindowOperator<Integer, Integer> op =
new AggregatingProcessingTimeWindowOperator<>(sumFunction, identitySelector, oneYear, oneYear);
-
- op.setup(out, mockContext);
- op.open(new Configuration());
+
+ op.setup(mockTask, new StreamConfig(new Configuration()), out);
+ op.open();
List<Integer> data = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10);
for (Integer i : data) {
- op.processElement(new StreamRecord<Integer>(i));
+ synchronized (lock) {
+ op.processElement(new StreamRecord<Integer>(i));
+ }
}
op.close();
@@ -466,15 +436,18 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
e.printStackTrace();
fail(e.getMessage());
}
+ finally {
+ timerService.shutdown();
+ }
}
@Test
public void testPropagateExceptionsFromProcessElement() {
+ final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
try {
final CollectingOutput<Integer> out = new CollectingOutput<>();
-
- final StreamingRuntimeContext mockContext = mock(StreamingRuntimeContext.class);
- when(mockContext.getTaskName()).thenReturn("Test task name");
+ final Object lock = new Object();
+ final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
ReduceFunction<Integer> failingFunction = new FailingFunction(100);
@@ -484,11 +457,13 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
new AggregatingProcessingTimeWindowOperator<>(
failingFunction, identitySelector, hundredYears, hundredYears);
- op.setup(out, mockContext);
- op.open(new Configuration());
+ op.setup(mockTask, new StreamConfig(new Configuration()), out);
+ op.open();
for (int i = 0; i < 100; i++) {
- op.processElement(new StreamRecord<Integer>(1));
+ synchronized (lock) {
+ op.processElement(new StreamRecord<Integer>(1));
+ }
}
try {
@@ -505,6 +480,9 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
e.printStackTrace();
fail(e.getMessage());
}
+ finally {
+ timerService.shutdown();
+ }
}
// ------------------------------------------------------------------------
@@ -546,4 +524,49 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
return value1 + value2;
}
}
+
+ private static StreamTask<?, ?> createMockTask() {
+ StreamTask<?, ?> task = mock(StreamTask.class);
+ when(task.getAccumulatorMap()).thenReturn(new HashMap<String, Accumulator<?, ?>>());
+ when(task.getName()).thenReturn("Test task name");
+ when(task.getExecutionConfig()).thenReturn(new ExecutionConfig());
+
+ Environment env = mock(Environment.class);
+ when(env.getIndexInSubtaskGroup()).thenReturn(0);
+ when(env.getNumberOfSubtasks()).thenReturn(1);
+ when(env.getUserClassLoader()).thenReturn(AggregatingAlignedProcessingTimeWindowOperatorTest.class.getClassLoader());
+
+ when(task.getEnvironment()).thenReturn(env);
+
+ return task;
+ }
+
+ private static StreamTask<?, ?> createMockTaskWithTimer(
+ final ScheduledExecutorService timerService, final Object lock)
+ {
+ StreamTask<?, ?> mockTask = createMockTask();
+
+ doAnswer(new Answer<Void>() {
+ @Override
+ public Void answer(InvocationOnMock invocationOnMock) throws Throwable {
+ final Long timestamp = (Long) invocationOnMock.getArguments()[0];
+ final Triggerable target = (Triggerable) invocationOnMock.getArguments()[1];
+ timerService.schedule(
+ new Callable<Object>() {
+ @Override
+ public Object call() throws Exception {
+ synchronized (lock) {
+ target.trigger(timestamp);
+ }
+ return null;
+ }
+ },
+ timestamp - System.currentTimeMillis(),
+ TimeUnit.MILLISECONDS);
+ return null;
+ }
+ }).when(mockTask).registerTimer(anyLong(), any(Triggerable.class));
+
+ return mockTask;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
index 06fca6b..6c48668 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTestHarness.java
@@ -142,7 +142,7 @@ public class StreamTaskTestHarness<OUT> {
outEdgesInOrder.add(new StreamEdge(sourceVertexDummy, targetVertexDummy, 0, new LinkedList<String>(), new BroadcastPartitioner<Object>()));
streamConfig.setOutEdgesInOrder(outEdgesInOrder);
streamConfig.setNonChainedOutputs(outEdgesInOrder);
- streamConfig.setTypeSerializerOut1(outputSerializer);
+ streamConfig.setTypeSerializerOut(outputSerializer);
streamConfig.setVertexID(0);
}
@@ -243,8 +243,8 @@ public class StreamTaskTestHarness<OUT> {
// first wait for all input queues to be empty
try {
Thread.sleep(1);
- } catch (InterruptedException e) {
- }
+ } catch (InterruptedException ignored) {}
+
while (true) {
boolean allEmpty = true;
for (int i = 0; i < numInputGates; i++) {
@@ -254,8 +254,8 @@ public class StreamTaskTestHarness<OUT> {
}
try {
Thread.sleep(10);
- } catch (InterruptedException e) {
- }
+ } catch (InterruptedException ignored) {}
+
if (allEmpty) {
break;
}
@@ -273,8 +273,7 @@ public class StreamTaskTestHarness<OUT> {
try {
Thread.sleep(1);
- } catch (InterruptedException e) {
- }
+ } catch (InterruptedException ignored) {}
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTimerITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTimerITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTimerITCase.java
index 7a53ceb..cdc2c53 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTimerITCase.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTimerITCase.java
@@ -23,6 +23,7 @@ import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.operators.StreamOperator;
import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
@@ -59,7 +60,7 @@ public class StreamTaskTimerITCase extends StreamingMultipleProgramsTestBase {
DataStream<String> source = env.addSource(new InfiniteTestSource());
- source.transform("Custom Operator", BasicTypeInfo.STRING_TYPE_INFO, new TimerOperator(StreamOperator.ChainingStrategy.ALWAYS));
+ source.transform("Custom Operator", BasicTypeInfo.STRING_TYPE_INFO, new TimerOperator(ChainingStrategy.ALWAYS));
boolean testSuccess = false;
try {
@@ -95,7 +96,7 @@ public class StreamTaskTimerITCase extends StreamingMultipleProgramsTestBase {
DataStream<String> source = env.addSource(new InfiniteTestSource());
- source.transform("Custom Operator", BasicTypeInfo.STRING_TYPE_INFO, new TimerOperator(StreamOperator.ChainingStrategy.NEVER));
+ source.transform("Custom Operator", BasicTypeInfo.STRING_TYPE_INFO, new TimerOperator(ChainingStrategy.NEVER));
boolean testSuccess = false;
try {
@@ -134,7 +135,7 @@ public class StreamTaskTimerITCase extends StreamingMultipleProgramsTestBase {
source.connect(source).transform(
"Custom Operator",
BasicTypeInfo.STRING_TYPE_INFO,
- new TwoInputTimerOperator(StreamOperator.ChainingStrategy.NEVER));
+ new TwoInputTimerOperator(ChainingStrategy.NEVER));
boolean testSuccess = false;
try {
@@ -180,7 +181,7 @@ public class StreamTaskTimerITCase extends StreamingMultipleProgramsTestBase {
}
if (first) {
- getRuntimeContext().registerTimer(System.currentTimeMillis() + 100, this);
+ registerTimer(System.currentTimeMillis() + 100, this);
first = false;
}
numElements++;
@@ -197,7 +198,7 @@ public class StreamTaskTimerITCase extends StreamingMultipleProgramsTestBase {
try {
numTimers++;
throwIfDone();
- getRuntimeContext().registerTimer(System.currentTimeMillis() + 1, this);
+ registerTimer(System.currentTimeMillis() + 1, this);
} finally {
semaphore.release();
}
@@ -236,7 +237,7 @@ public class StreamTaskTimerITCase extends StreamingMultipleProgramsTestBase {
}
if (first) {
- getRuntimeContext().registerTimer(System.currentTimeMillis() + 100, this);
+ registerTimer(System.currentTimeMillis() + 100, this);
first = false;
}
numElements++;
@@ -251,7 +252,7 @@ public class StreamTaskTimerITCase extends StreamingMultipleProgramsTestBase {
}
if (first) {
- getRuntimeContext().registerTimer(System.currentTimeMillis() + 100, this);
+ registerTimer(System.currentTimeMillis() + 100, this);
first = false;
}
numElements++;
@@ -269,7 +270,7 @@ public class StreamTaskTimerITCase extends StreamingMultipleProgramsTestBase {
try {
numTimers++;
throwIfDone();
- getRuntimeContext().registerTimer(System.currentTimeMillis() + 1, this);
+ registerTimer(System.currentTimeMillis() + 1, this);
} finally {
semaphore.release();
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java
index a88aa1a..dafba9c 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/timestamp/TimestampITCase.java
@@ -439,8 +439,8 @@ public class TimestampITCase {
}
@Override
- public void open(Configuration parameters) throws Exception {
- super.open(parameters);
+ public void open() throws Exception {
+ super.open();
watermarks = new ArrayList<Watermark>();
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java
index 3651230..000a1a2 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java
@@ -21,17 +21,33 @@ import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.accumulators.Accumulator;
+import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.operators.testutils.MockEnvironment;
import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;
+import org.apache.flink.streaming.api.graph.StreamConfig;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.runtime.operators.Triggerable;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Matchers.anyLong;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
public class MockContext<IN, OUT> {
+
private Collection<IN> inputs;
private List<OUT> outputs;
@@ -57,27 +73,63 @@ public class MockContext<IN, OUT> {
public static <IN, OUT> List<OUT> createAndExecute(OneInputStreamOperator<IN, OUT> operator, List<IN> inputs) {
MockContext<IN, OUT> mockContext = new MockContext<IN, OUT>(inputs);
- StreamingRuntimeContext runtimeContext = new StreamingRuntimeContext(
- new MockEnvironment("MockTask", 3 * 1024 * 1024, new MockInputSplitProvider(), 1024),
- new ExecutionConfig(),
- null, null,
- new HashMap<String, Accumulator<?, ?>>(),
- null);
-
- operator.setup(mockContext.output, runtimeContext);
+
+ final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
+ final Object lock = new Object();
+ final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
+
+ operator.setup(mockTask, new StreamConfig(new Configuration()), mockContext.output);
try {
- operator.open(null);
+ operator.open();
StreamRecord<IN> nextRecord;
for (IN in: inputs) {
- operator.processElement(new StreamRecord<IN>(in));
+ synchronized (lock) {
+ operator.processElement(new StreamRecord<IN>(in));
+ }
}
operator.close();
} catch (Exception e) {
throw new RuntimeException("Cannot invoke operator.", e);
+ } finally {
+ timerService.shutdownNow();
}
return mockContext.getOutputs();
}
+
+ private static StreamTask<?, ?> createMockTaskWithTimer(
+ final ScheduledExecutorService timerService, final Object lock)
+ {
+ StreamTask<?, ?> task = mock(StreamTask.class);
+ when(task.getAccumulatorMap()).thenReturn(new HashMap<String, Accumulator<?, ?>>());
+ when(task.getName()).thenReturn("Test task name");
+ when(task.getExecutionConfig()).thenReturn(new ExecutionConfig());
+ when(task.getEnvironment()).thenReturn(new MockEnvironment("MockTask", 3 * 1024 * 1024, new MockInputSplitProvider(), 1024));
+ when(task.getCheckpointLock()).thenReturn(lock);
+
+ doAnswer(new Answer<Void>() {
+ @Override
+ public Void answer(InvocationOnMock invocationOnMock) throws Throwable {
+ final Long timestamp = (Long) invocationOnMock.getArguments()[0];
+ final Triggerable target = (Triggerable) invocationOnMock.getArguments()[1];
+ timerService.schedule(
+ new Callable<Object>() {
+ @Override
+ public Object call() throws Exception {
+ synchronized (lock) {
+ target.trigger(timestamp);
+ }
+ return null;
+ }
+ },
+ timestamp - System.currentTimeMillis(),
+ TimeUnit.MILLISECONDS);
+ return null;
+ }
+ }).when(task).registerTimer(anyLong(), any(Triggerable.class));
+
+ return task;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
index f5ce3fc..edf3a09 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
@@ -18,25 +18,28 @@
package org.apache.flink.streaming.util;
import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.accumulators.Accumulator;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.java.typeutils.TypeExtractor;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.execution.Environment;
import org.apache.flink.runtime.operators.testutils.MockEnvironment;
import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;
-import org.apache.flink.runtime.state.LocalStateHandle;
+import org.apache.flink.streaming.api.graph.StreamConfig;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.state.StateBackend;
+import org.apache.flink.streaming.api.state.memory.MemoryStateBackend;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask;
-import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.mockito.stubbing.OngoingStubbing;
-import java.io.Serializable;
import java.util.Collection;
-import java.util.HashMap;
import java.util.concurrent.ConcurrentLinkedQueue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
/**
* A test harness for testing a {@link OneInputStreamOperator}.
*
@@ -47,28 +50,39 @@ import java.util.concurrent.ConcurrentLinkedQueue;
*/
public class OneInputStreamOperatorTestHarness<IN, OUT> {
- OneInputStreamOperator<IN, OUT> operator;
+ final OneInputStreamOperator<IN, OUT> operator;
- ConcurrentLinkedQueue<Object> outputList;
+ final ConcurrentLinkedQueue<Object> outputList;
- ExecutionConfig executionConfig;
+ final ExecutionConfig executionConfig;
+
+ final Object checkpointLock;
public OneInputStreamOperatorTestHarness(OneInputStreamOperator<IN, OUT> operator) {
+ this(operator, new StreamConfig(new Configuration()));
+ }
+
+ public OneInputStreamOperatorTestHarness(OneInputStreamOperator<IN, OUT> operator, StreamConfig config) {
this.operator = operator;
+ this.outputList = new ConcurrentLinkedQueue<Object>();
+ this.executionConfig = new ExecutionConfig();
+ this.checkpointLock = new Object();
+
+ Environment env = new MockEnvironment("MockTwoInputTask", 3 * 1024 * 1024, new MockInputSplitProvider(), 1024);
+ StreamTask<?, ?> mockTask = mock(StreamTask.class);
+ when(mockTask.getName()).thenReturn("Mock Task");
+ when(mockTask.getCheckpointLock()).thenReturn(checkpointLock);
+ when(mockTask.getConfiguration()).thenReturn(config);
+ when(mockTask.getEnvironment()).thenReturn(env);
+ when(mockTask.getExecutionConfig()).thenReturn(executionConfig);
+
+ // ugly Java generic hacks
+ @SuppressWarnings("unchecked")
+ OngoingStubbing<StateBackend<?>> stubbing =
+ (OngoingStubbing<StateBackend<?>>) (OngoingStubbing<?>) when(mockTask.getStateBackend());
+ stubbing.thenReturn(MemoryStateBackend.defaultInstance());
- outputList = new ConcurrentLinkedQueue<Object>();
-
- executionConfig = new ExecutionConfig();
-
- StreamingRuntimeContext runtimeContext = new StreamingRuntimeContext(
- new MockEnvironment("MockTwoInputTask", 3 * 1024 * 1024, new MockInputSplitProvider(), 1024),
- executionConfig,
- null,
- new LocalStateHandle.LocalStateHandleProvider<Serializable>(),
- new HashMap<String, Accumulator<?, ?>>(),
- new OneInputStreamTask());
-
- operator.setup(new MockOutput(), runtimeContext);
+ operator.setup(mockTask, new StreamConfig(new Configuration()), new MockOutput());
}
/**
@@ -81,19 +95,10 @@ public class OneInputStreamOperatorTestHarness<IN, OUT> {
}
/**
- * Calls {@link org.apache.flink.streaming.api.operators.StreamOperator#open(org.apache.flink.configuration.Configuration)}
- * with an empty {@link org.apache.flink.configuration.Configuration}.
+ * Calls {@link org.apache.flink.streaming.api.operators.StreamOperator#open()}
*/
public void open() throws Exception {
- operator.open(new Configuration());
- }
-
- /**
- * Calls {@link org.apache.flink.streaming.api.operators.StreamOperator#open(org.apache.flink.configuration.Configuration)}
- * with the given {@link org.apache.flink.configuration.Configuration}.
- */
- public void open(Configuration config) throws Exception {
- operator.open(config);
+ operator.open();
}
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/SourceFunctionUtil.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/SourceFunctionUtil.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/SourceFunctionUtil.java
index 428131a..2afdc40 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/SourceFunctionUtil.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/SourceFunctionUtil.java
@@ -17,7 +17,6 @@
package org.apache.flink.streaming.util;
-import java.io.Serializable;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
@@ -29,26 +28,30 @@ import org.apache.flink.api.common.functions.RuntimeContext;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.operators.testutils.MockEnvironment;
import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;
-import org.apache.flink.runtime.state.LocalStateHandle;
import org.apache.flink.streaming.api.functions.source.EventTimeSourceFunction;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
import org.apache.flink.streaming.api.operators.Output;
import org.apache.flink.streaming.api.operators.StreamSource;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+
+import static org.mockito.Mockito.*;
public class SourceFunctionUtil<T> {
public static <T> List<T> runSourceFunction(SourceFunction<T> sourceFunction) throws Exception {
List<T> outputs = new ArrayList<T>();
+
if (sourceFunction instanceof RichFunction) {
+
+ AbstractStreamOperator<?> operator = mock(AbstractStreamOperator.class);
+ when(operator.getExecutionConfig()).thenReturn(new ExecutionConfig());
+
RuntimeContext runtimeContext = new StreamingRuntimeContext(
+ operator,
new MockEnvironment("MockTask", 3 * 1024 * 1024, new MockInputSplitProvider(), 1024),
- new ExecutionConfig(),
- null,
- new LocalStateHandle.LocalStateHandleProvider<Serializable>(),
- new HashMap<String, Accumulator<?, ?>>(),
- null);
+ new HashMap<String, Accumulator<?, ?>>());
((RichFunction) sourceFunction).setRuntimeContext(runtimeContext);
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TwoInputStreamOperatorTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TwoInputStreamOperatorTestHarness.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TwoInputStreamOperatorTestHarness.java
index 2418f19..9b33c6a 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TwoInputStreamOperatorTestHarness.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/TwoInputStreamOperatorTestHarness.java
@@ -19,24 +19,27 @@
package org.apache.flink.streaming.util;
import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.accumulators.Accumulator;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.java.typeutils.TypeExtractor;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.execution.Environment;
import org.apache.flink.runtime.operators.testutils.MockEnvironment;
import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;
-import org.apache.flink.runtime.state.LocalStateHandle;
+import org.apache.flink.streaming.api.graph.StreamConfig;
import org.apache.flink.streaming.api.operators.Output;
import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
+import org.apache.flink.streaming.api.state.StateBackend;
+import org.apache.flink.streaming.api.state.memory.MemoryStateBackend;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext;
-import org.apache.flink.streaming.runtime.tasks.TwoInputStreamTask;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.mockito.stubbing.OngoingStubbing;
-import java.io.Serializable;
-import java.util.HashMap;
import java.util.concurrent.ConcurrentLinkedQueue;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
/**
* A test harness for testing a {@link TwoInputStreamOperator}.
*
@@ -49,26 +52,37 @@ public class TwoInputStreamOperatorTestHarness<IN1, IN2, OUT> {
TwoInputStreamOperator<IN1, IN2, OUT> operator;
- ConcurrentLinkedQueue<Object> outputList;
+ final ConcurrentLinkedQueue<Object> outputList;
+
+ final ExecutionConfig executionConfig;
- ExecutionConfig executionConfig;
+ final Object checkpointLock;
public TwoInputStreamOperatorTestHarness(TwoInputStreamOperator<IN1, IN2, OUT> operator) {
+ this(operator, new StreamConfig(new Configuration()));
+ }
+
+ public TwoInputStreamOperatorTestHarness(TwoInputStreamOperator<IN1, IN2, OUT> operator, StreamConfig config) {
this.operator = operator;
+ this.outputList = new ConcurrentLinkedQueue<Object>();
+ this.executionConfig = new ExecutionConfig();
+ this.checkpointLock = new Object();
+
+ Environment env = new MockEnvironment("MockTwoInputTask", 3 * 1024 * 1024, new MockInputSplitProvider(), 1024);
+ StreamTask<?, ?> mockTask = mock(StreamTask.class);
+ when(mockTask.getName()).thenReturn("Mock Task");
+ when(mockTask.getCheckpointLock()).thenReturn(checkpointLock);
+ when(mockTask.getConfiguration()).thenReturn(config);
+ when(mockTask.getEnvironment()).thenReturn(env);
+ when(mockTask.getExecutionConfig()).thenReturn(executionConfig);
+
+ // ugly Java generic hacks
+ @SuppressWarnings("unchecked")
+ OngoingStubbing<StateBackend<?>> stubbing =
+ (OngoingStubbing<StateBackend<?>>) (OngoingStubbing<?>) when(mockTask.getStateBackend());
+ stubbing.thenReturn(MemoryStateBackend.defaultInstance());
- outputList = new ConcurrentLinkedQueue<Object>();
-
- executionConfig = new ExecutionConfig();
-
- StreamingRuntimeContext runtimeContext = new StreamingRuntimeContext(
- new MockEnvironment("MockTwoInputTask", 3 * 1024 * 1024, new MockInputSplitProvider(), 1024),
- new ExecutionConfig(),
- null,
- new LocalStateHandle.LocalStateHandleProvider<>(),
- new HashMap<String, Accumulator<?, ?>>(),
- new TwoInputStreamTask());
-
- operator.setup(new MockOutput(), runtimeContext);
+ operator.setup(mockTask, new StreamConfig(new Configuration()), new MockOutput());
}
/**
@@ -82,19 +96,10 @@ public class TwoInputStreamOperatorTestHarness<IN1, IN2, OUT> {
/**
- * Calls {@link org.apache.flink.streaming.api.operators.StreamOperator#open(Configuration)}
- * with an empty {@link Configuration}.
+ * Calls {@link org.apache.flink.streaming.api.operators.StreamOperator#open()}.
*/
public void open() throws Exception {
- operator.open(new Configuration());
- }
-
- /**
- * Calls {@link org.apache.flink.streaming.api.operators.StreamOperator#open(Configuration)}
- * with the given {@link Configuration}.
- */
- public void open(Configuration config) throws Exception {
- operator.open(config);
+ operator.open();
}
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
index 0565f52..6855e00 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/DataStream.scala
@@ -228,11 +228,13 @@ class DataStream[T](javaStream: JavaStream[T]) {
def keyBy[K: TypeInformation](fun: T => K): KeyedStream[T, K] = {
val cleanFun = clean(fun)
+ val keyType: TypeInformation[K] = implicitly[TypeInformation[K]]
+
val keyExtractor = new KeySelector[T, K] with ResultTypeQueryable[K] {
def getKey(in: T) = cleanFun(in)
- override def getProducedType: TypeInformation[K] = implicitly[TypeInformation[K]]
+ override def getProducedType: TypeInformation[K] = keyType
}
- javaStream.keyBy(keyExtractor)
+ new JavaKeyedStream(javaStream, keyExtractor, keyType)
}
/**
@@ -431,32 +433,7 @@ class DataStream[T](javaStream: JavaStream[T]) {
val outType : TypeInformation[R] = implicitly[TypeInformation[R]]
javaStream.map(mapper).returns(outType).asInstanceOf[JavaStream[R]]
}
-
- /**
- * Creates a new DataStream by applying the given stateful function to every element of this
- * DataStream. To use state partitioning, a key must be defined using .keyBy(..), in which
- * case an independent state will be kept per key.
- *
- * Note that the user state object needs to be serializable.
- */
- def mapWithState[R: TypeInformation: ClassTag, S](
- fun: (T, Option[S]) => (R, Option[S])): DataStream[R] = {
- if (fun == null) {
- throw new NullPointerException("Map function must not be null.")
- }
-
- val cleanFun = clean(fun)
- val mapper = new RichMapFunction[T, R] with StatefulFunction[T, R, S] {
- override def map(in: T): R = {
- applyWithState(in, cleanFun)
- }
-
- val partitioned = isStatePartitioned
- }
-
- map(mapper)
- }
-
+
/**
* Creates a new DataStream by applying the given function to every element and flattening
* the results.
@@ -501,32 +478,6 @@ class DataStream[T](javaStream: JavaStream[T]) {
}
/**
- * Creates a new DataStream by applying the given stateful function to every element and
- * flattening the results. To use state partitioning, a key must be defined using .keyBy(..),
- * in which case an independent state will be kept per key.
- *
- * Note that the user state object needs to be serializable.
- */
- def flatMapWithState[R: TypeInformation: ClassTag, S](
- fun: (T, Option[S]) => (TraversableOnce[R], Option[S])):
- DataStream[R] = {
- if (fun == null) {
- throw new NullPointerException("Flatmap function must not be null.")
- }
-
- val cleanFun = clean(fun)
- val flatMapper = new RichFlatMapFunction[T, R] with StatefulFunction[T,TraversableOnce[R],S]{
- override def flatMap(in: T, out: Collector[R]): Unit = {
- applyWithState(in, cleanFun) foreach out.collect
- }
-
- val partitioned = isStatePartitioned
- }
-
- flatMap(flatMapper)
- }
-
- /**
* Creates a new DataStream that contains only the elements satisfying the given filter predicate.
*/
def filter(filter: FilterFunction[T]): DataStream[T] = {
@@ -549,35 +500,6 @@ class DataStream[T](javaStream: JavaStream[T]) {
}
this.filter(filter)
}
-
- /**
- * Creates a new DataStream that contains only the elements satisfying the given stateful filter
- * predicate. To use state partitioning, a key must be defined using .keyBy(..), in which case
- * an independent state will be kept per key.
- *
- * Note that the user state object needs to be serializable.
- */
- def filterWithState[S](
- fun: (T, Option[S]) => (Boolean, Option[S])): DataStream[T] = {
- if (fun == null) {
- throw new NullPointerException("Filter function must not be null.")
- }
-
- val cleanFun = clean(fun)
- val filterFun = new RichFilterFunction[T] with StatefulFunction[T, Boolean, S] {
- override def filter(in: T): Boolean = {
- applyWithState(in, cleanFun)
- }
-
- val partitioned = isStatePartitioned
- }
-
- filter(filterFun)
- }
-
- private[flink] def isStatePartitioned: Boolean = {
- javaStream.isInstanceOf[JavaKeyedStream[_, _]]
- }
/**
* Windows this DataStream into tumbling time windows.
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala
index a588931..84354a3 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala
@@ -18,18 +18,19 @@
package org.apache.flink.streaming.api.scala
-import org.apache.flink.streaming.api.datastream.{KeyedStream => KeyedJavaStream, DataStream => JavaStream, WindowedStream => WindowedJavaStream}
+import org.apache.flink.api.common.functions._
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.streaming.api.datastream.{DataStream => JavaStream, KeyedStream => KeyedJavaStream, WindowedStream => WindowedJavaStream}
import org.apache.flink.streaming.api.functions.aggregation.AggregationFunction.AggregationType
-import org.apache.flink.streaming.api.functions.aggregation.SumAggregator
-import org.apache.flink.streaming.api.functions.aggregation.ComparableAggregator
+import org.apache.flink.streaming.api.functions.aggregation.{ComparableAggregator, SumAggregator}
import org.apache.flink.streaming.api.operators.StreamGroupedReduce
+import org.apache.flink.streaming.api.scala.function.StatefulFunction
import org.apache.flink.streaming.api.windowing.assigners._
import org.apache.flink.streaming.api.windowing.time.AbstractTime
-import org.apache.flink.streaming.api.windowing.windows.{GlobalWindow, Window, TimeWindow}
+import org.apache.flink.streaming.api.windowing.windows.{GlobalWindow, TimeWindow, Window}
+import org.apache.flink.util.Collector
+
import scala.reflect.ClassTag
-import org.apache.flink.api.common.typeinfo.TypeInformation
-import org.apache.flink.api.common.functions.FoldFunction
-import org.apache.flink.api.common.functions.ReduceFunction
class KeyedStream[T, K](javaStream: KeyedJavaStream[T, K]) extends DataStream[T](javaStream) {
@@ -262,10 +263,99 @@ class KeyedStream[T, K](javaStream: KeyedJavaStream[T, K]) extends DataStream[T]
javaStream.getExecutionConfig)
}
- val invokable = new StreamGroupedReduce[T](reducer,javaStream.getKeySelector(),getType())
+ val invokable = new StreamGroupedReduce[T](reducer,
+ getType().createSerializer(getExecutionConfig))
new DataStream[T](javaStream.transform("aggregation", javaStream.getType(),invokable))
.asInstanceOf[DataStream[T]]
}
+
+ // ------------------------------------------------------------------------
+ // functions with state
+ // ------------------------------------------------------------------------
+
+ /**
+ * Creates a new DataStream that contains only the elements satisfying the given stateful filter
+ * predicate. To use state partitioning, a key must be defined using .keyBy(..), in which case
+ * an independent state will be kept per key.
+ *
+ * Note that the user state object needs to be serializable.
+ */
+ def filterWithState[S : TypeInformation](
+ fun: (T, Option[S]) => (Boolean, Option[S])): DataStream[T] = {
+ if (fun == null) {
+ throw new NullPointerException("Filter function must not be null.")
+ }
+
+ val cleanFun = clean(fun)
+ val stateTypeInfo: TypeInformation[S] = implicitly[TypeInformation[S]]
+
+ val filterFun = new RichFilterFunction[T] with StatefulFunction[T, Boolean, S] {
+
+ override val stateType: TypeInformation[S] = stateTypeInfo
+
+ override def filter(in: T): Boolean = {
+ applyWithState(in, cleanFun)
+ }
+ }
+
+ filter(filterFun)
+ }
+
+ /**
+ * Creates a new DataStream by applying the given stateful function to every element of this
+ * DataStream. To use state partitioning, a key must be defined using .keyBy(..), in which
+ * case an independent state will be kept per key.
+ *
+ * Note that the user state object needs to be serializable.
+ */
+ def mapWithState[R: TypeInformation: ClassTag, S: TypeInformation](
+ fun: (T, Option[S]) => (R, Option[S])): DataStream[R] = {
+ if (fun == null) {
+ throw new NullPointerException("Map function must not be null.")
+ }
+
+ val cleanFun = clean(fun)
+ val stateTypeInfo: TypeInformation[S] = implicitly[TypeInformation[S]]
+
+ val mapper = new RichMapFunction[T, R] with StatefulFunction[T, R, S] {
+
+ override val stateType: TypeInformation[S] = stateTypeInfo
+
+ override def map(in: T): R = {
+ applyWithState(in, cleanFun)
+ }
+ }
+
+ map(mapper)
+ }
+
+ /**
+ * Creates a new DataStream by applying the given stateful function to every element and
+ * flattening the results. To use state partitioning, a key must be defined using .keyBy(..),
+ * in which case an independent state will be kept per key.
+ *
+ * Note that the user state object needs to be serializable.
+ */
+ def flatMapWithState[R: TypeInformation: ClassTag, S: TypeInformation](
+ fun: (T, Option[S]) => (TraversableOnce[R], Option[S])): DataStream[R] = {
+ if (fun == null) {
+ throw new NullPointerException("Flatmap function must not be null.")
+ }
+
+ val cleanFun = clean(fun)
+ val stateTypeInfo: TypeInformation[S] = implicitly[TypeInformation[S]]
+
+ val flatMapper = new RichFlatMapFunction[T, R] with StatefulFunction[T,TraversableOnce[R],S]{
+
+ override val stateType: TypeInformation[S] = stateTypeInfo
+
+ override def flatMap(in: T, out: Collector[R]): Unit = {
+ applyWithState(in, cleanFun) foreach out.collect
+ }
+ }
+
+ flatMap(flatMapper)
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
index f767aba..29bf938 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/StreamExecutionEnvironment.scala
@@ -19,13 +19,14 @@
package org.apache.flink.streaming.api.scala
import java.util.Objects
+import java.util.Objects._
import com.esotericsoftware.kryo.Serializer
import org.apache.flink.api.common.io.{FileInputFormat, InputFormat}
import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer
import org.apache.flink.api.scala.ClosureCleaner
-import org.apache.flink.runtime.state.StateHandleProvider
+import org.apache.flink.streaming.api.state.StateBackend
import org.apache.flink.streaming.api.{TimeCharacteristic, CheckpointingMode}
import org.apache.flink.streaming.api.environment.{StreamExecutionEnvironment => JavaEnv}
import org.apache.flink.streaming.api.functions.source.FileMonitoringFunction.WatchType
@@ -184,17 +185,39 @@ class StreamExecutionEnvironment(javaEnv: JavaEnv) {
javaEnv.enableCheckpointing()
this
}
-
- /**
- * Sets the given StateHandleProvider to be used for storing operator state
- * checkpoints when checkpointing is enabled.
- */
- def setStateHandleProvider(provider: StateHandleProvider[_]): StreamExecutionEnvironment = {
- javaEnv.setStateHandleProvider(provider)
+
+ def getCheckpointingMode = javaEnv.getCheckpointingMode()
+
+ /**
+ * Sets the state backend that describes how to store and checkpoint operator state.
+ * It defines in what form the key/value state, accessible from operations on
+ * [[KeyedStream]] is maintained (heap, managed memory, externally), and where state
+ * snapshots/checkpoints are stored, both for the key/value state, and for checkpointed
+ * functions (implementing the interface
+ * [[org.apache.flink.streaming.api.checkpoint.Checkpointed]].
+ *
+ * <p>The [[org.apache.flink.streaming.api.state.memory.MemoryStateBackend]] for example
+ * maintains the state in heap memory, as objects. It is lightweight without extra
+ * dependencies, but can checkpoint only small states (some counters).
+ *
+ * <p>In contrast, the [[org.apache.flink.streaming.api.state.filesystem.FsStateBackend]]
+ * stores checkpoints of the state (also maintained as heap objects) in files. When using
+ * a replicated file system (like HDFS, S3, MapR FS, Tachyon, etc) this will guarantee
+ * that state is not lost upon failures of individual nodes and that the entire streaming
+ * program can be executed highly available and strongly consistent (assuming that Flink
+ * is run in high-availability mode).
+ */
+ def setStateBackend(backend: StateBackend[_]): StreamExecutionEnvironment = {
+ javaEnv.setStateBackend(backend)
this
}
/**
+ * Returns the state backend that defines how to store and checkpoint state.
+ */
+ def getStateBackend: StateBackend[_] = javaEnv.getStateBackend()
+
+ /**
* Sets the number of times that failed tasks are re-executed. A value of zero
* effectively disables fault tolerance. A value of "-1" indicates that the system
* default value (as defined in the configuration) should be used.
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/StatefulFunction.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/StatefulFunction.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/StatefulFunction.scala
index 89c9d00..5a591a8 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/StatefulFunction.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/StatefulFunction.scala
@@ -19,6 +19,7 @@
package org.apache.flink.streaming.api.scala.function
import org.apache.flink.api.common.functions.RichFunction
+import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.configuration.Configuration
import org.apache.flink.api.common.state.OperatorState
@@ -28,17 +29,20 @@ import org.apache.flink.api.common.state.OperatorState
* call the applyWithState method in his own RichFunction implementation.
*/
trait StatefulFunction[I, O, S] extends RichFunction {
-
- var state: OperatorState[Option[S]] = _
- val partitioned: Boolean
+
+ var state: OperatorState[S] = _
+ val stateType: TypeInformation[S]
def applyWithState(in: I, fun: (I, Option[S]) => (O, Option[S])): O = {
- val (o, s) = fun(in, state.value)
- state.update(s)
+ val (o, s: Option[S]) = fun(in, Option(state.value()))
+ s match {
+ case Some(v) => state.update(v)
+ case None => state.update(null.asInstanceOf[S])
+ }
o
}
override def open(c: Configuration) = {
- state = getRuntimeContext().getOperatorState("state", None, partitioned)
+ state = getRuntimeContext().getKeyValueState[S](stateType, null.asInstanceOf[S])
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala
index 91639ed..fe85fd1 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala
@@ -19,7 +19,7 @@
package org.apache.flink.streaming.api.scala
import java.lang
-import org.apache.flink.api.common.functions.{FilterFunction, FlatMapFunction, MapFunction, Partitioner, FoldFunction, Function}
+import org.apache.flink.api.common.functions._
import org.apache.flink.api.java.typeutils.TypeExtractor
import org.apache.flink.streaming.api.collector.selector.OutputSelector
import org.apache.flink.streaming.api.functions.co.CoMapFunction
@@ -28,12 +28,13 @@ import org.apache.flink.streaming.api.operators.{AbstractUdfStreamOperator, Stre
import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows
import org.apache.flink.streaming.api.windowing.triggers.{PurgingTrigger, CountTrigger}
import org.apache.flink.streaming.api.windowing.windows.GlobalWindow
+import org.apache.flink.streaming.api.scala._
import org.apache.flink.streaming.runtime.partitioner._
import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase
import org.apache.flink.util.Collector
+
import org.junit.Assert.fail
import org.junit.Test
-import org.apache.flink.streaming.api.scala.function.StatefulFunction
class DataStreamTest extends StreamingMultipleProgramsTestBase {
@@ -239,7 +240,7 @@ class DataStreamTest extends StreamingMultipleProgramsTestBase {
* Tests whether parallelism gets set.
*/
@Test
- def testParallelism {
+ def testParallelism() {
val env: StreamExecutionEnvironment = StreamExecutionEnvironment.createLocalEnvironment(10)
val src = env.fromElements(new Tuple2[Long, Long](0L, 0L))
@@ -259,7 +260,7 @@ class DataStreamTest extends StreamingMultipleProgramsTestBase {
try {
src.setParallelism(3)
- fail
+ fail()
}
catch {
case success: IllegalArgumentException => {
@@ -290,14 +291,14 @@ class DataStreamTest extends StreamingMultipleProgramsTestBase {
}
@Test
- def testTypeInfo {
+ def testTypeInfo() {
val env = StreamExecutionEnvironment.getExecutionEnvironment
val src1: DataStream[Long] = env.generateSequence(0, 0)
assert(TypeExtractor.getForClass(classOf[Long]) == src1.getType)
val map: DataStream[(Integer, String)] = src1.map(x => null)
- assert(classOf[scala.Tuple2[Integer, String]] == map.getType.getTypeClass)
+ assert(classOf[scala.Tuple2[Integer, String]] == map.getType().getTypeClass)
val window: DataStream[String] = map
.windowAll(GlobalWindows.create())
@@ -310,12 +311,12 @@ class DataStreamTest extends StreamingMultipleProgramsTestBase {
.windowAll(GlobalWindows.create())
.trigger(PurgingTrigger.of(CountTrigger.of[GlobalWindow](5)))
.fold(0, (accumulator: Int, value: String) => 0)
- assert(TypeExtractor.getForClass(classOf[Int]) == flatten.getType)
+ assert(TypeExtractor.getForClass(classOf[Int]) == flatten.getType())
// TODO check for custom case class
}
- @Test def operatorTest {
+ @Test def operatorTest() {
val env = StreamExecutionEnvironment.getExecutionEnvironment
val src = env.generateSequence(0, 0)
@@ -327,20 +328,14 @@ class DataStreamTest extends StreamingMultipleProgramsTestBase {
val map = src.map(mapFunction)
assert(mapFunction == getFunctionForDataStream(map))
assert(getFunctionForDataStream(map.map(x => 0)).isInstanceOf[MapFunction[_, _]])
-
- val statefulMap1 = src.mapWithState((in, state: Option[Long]) => (in, None))
- assert(getFunctionForDataStream(statefulMap1).isInstanceOf[MapFunction[_,_]])
- assert(!getFunctionForDataStream(statefulMap1).
- asInstanceOf[StatefulFunction[_,_,_]].partitioned)
- val statefulMap2 = src.keyBy(x=>x).mapWithState(
- (in, state: Option[Long]) => (in, None))
- assert(getFunctionForDataStream(statefulMap2).
- asInstanceOf[StatefulFunction[_,_,_]].partitioned)
+ val statefulMap2 = src.keyBy(x => x).mapWithState(
+ (in, state: Option[Long]) => (in, None.asInstanceOf[Option[Long]]))
val flatMapFunction = new FlatMapFunction[Long, Int] {
override def flatMap(value: Long, out: Collector[Int]): Unit = {}
}
+
val flatMap = src.flatMap(flatMapFunction)
assert(flatMapFunction == getFunctionForDataStream(flatMap))
assert(
@@ -348,15 +343,8 @@ class DataStreamTest extends StreamingMultipleProgramsTestBase {
.flatMap((x: Int, out: Collector[Int]) => {}))
.isInstanceOf[FlatMapFunction[_, _]])
- val statefulfMap1 = src.flatMapWithState((in, state: Option[Long]) => (List(in), None))
- assert(getFunctionForDataStream(statefulfMap1).isInstanceOf[FlatMapFunction[_, _]])
- assert(!getFunctionForDataStream(statefulfMap1).
- asInstanceOf[StatefulFunction[_, _, _]].partitioned)
-
- val statefulfMap2 = src.keyBy(x=>x).flatMapWithState(
- (in, state: Option[Long]) => (List(in), None))
- assert(getFunctionForDataStream(statefulfMap2).
- asInstanceOf[StatefulFunction[_, _, _]].partitioned)
+ val statefulfMap2 = src.keyBy(x => x).flatMapWithState(
+ (in, state: Option[Long]) => (List(in), None.asInstanceOf[Option[Long]]))
val filterFunction = new FilterFunction[Int] {
override def filter(value: Int): Boolean = false
@@ -369,15 +357,8 @@ class DataStreamTest extends StreamingMultipleProgramsTestBase {
.filter((x: Int) => true))
.isInstanceOf[FilterFunction[_]])
- val statefulFilter1 = src.filterWithState((in, state: Option[Long]) => (true, None))
- assert(getFunctionForDataStream(statefulFilter1).isInstanceOf[FilterFunction[_]])
- assert(!getFunctionForDataStream(statefulFilter1).
- asInstanceOf[StatefulFunction[_, _, _]].partitioned)
-
- val statefulFilter2 = src.keyBy(x=>x).filterWithState(
+ val statefulFilter2 = src.keyBy( x => x).filterWithState[Long](
(in, state: Option[Long]) => (false, None))
- assert(getFunctionForDataStream(statefulFilter2).
- asInstanceOf[StatefulFunction[_, _, _]].partitioned)
try {
env.getStreamGraph.getStreamEdge(map.getId, unionFilter.getId)
@@ -412,7 +393,7 @@ class DataStreamTest extends StreamingMultipleProgramsTestBase {
assert(2 == moreOutputSelectors.size)
val select = split.select("a")
- val sink = select.print
+ val sink = select.print()
val splitEdge =
env.getStreamGraph.getStreamEdge(unionFilter.getId, sink.getTransformation.getId)
assert("a" == splitEdge.getSelectedNames.get(0))
@@ -457,44 +438,44 @@ class DataStreamTest extends StreamingMultipleProgramsTestBase {
}
@Test
- def testChannelSelectors {
+ def testChannelSelectors() {
val env = StreamExecutionEnvironment.getExecutionEnvironment
val src = env.generateSequence(0, 0)
val broadcast = src.broadcast
- val broadcastSink = broadcast.print
+ val broadcastSink = broadcast.print()
val broadcastPartitioner = env.getStreamGraph
.getStreamEdge(src.getId, broadcastSink.getTransformation.getId).getPartitioner
assert(broadcastPartitioner.isInstanceOf[BroadcastPartitioner[_]])
val shuffle: DataStream[Long] = src.shuffle
- val shuffleSink = shuffle.print
+ val shuffleSink = shuffle.print()
val shufflePartitioner = env.getStreamGraph
.getStreamEdge(src.getId, shuffleSink.getTransformation.getId).getPartitioner
assert(shufflePartitioner.isInstanceOf[ShufflePartitioner[_]])
val forward: DataStream[Long] = src.forward
- val forwardSink = forward.print
+ val forwardSink = forward.print()
val forwardPartitioner = env.getStreamGraph
.getStreamEdge(src.getId, forwardSink.getTransformation.getId).getPartitioner
assert(forwardPartitioner.isInstanceOf[ForwardPartitioner[_]])
val rebalance: DataStream[Long] = src.rebalance
- val rebalanceSink = rebalance.print
+ val rebalanceSink = rebalance.print()
val rebalancePartitioner = env.getStreamGraph
.getStreamEdge(src.getId, rebalanceSink.getTransformation.getId).getPartitioner
assert(rebalancePartitioner.isInstanceOf[RebalancePartitioner[_]])
val global: DataStream[Long] = src.global
- val globalSink = global.print
+ val globalSink = global.print()
val globalPartitioner = env.getStreamGraph
.getStreamEdge(src.getId, globalSink.getTransformation.getId).getPartitioner
assert(globalPartitioner.isInstanceOf[GlobalPartitioner[_]])
}
@Test
- def testIterations {
+ def testIterations() {
val env = StreamExecutionEnvironment.getExecutionEnvironment
// we need to rebalance before iteration
val source = env.fromElements(1, 2, 3).map { t: Int => t }
@@ -512,10 +493,10 @@ class DataStreamTest extends StreamingMultipleProgramsTestBase {
val head = input.partitionByHash(1, 1).map(i => (i + 1).toString, s => s)
(head.filter(_ == "2"), head.filter(_ != "2"))
}, 1000).print()
- fail
+ fail()
} catch {
case uoe: UnsupportedOperationException =>
- case e: Exception => fail
+ case e: Exception => fail()
}
val sg = env.getStreamGraph
@@ -531,7 +512,7 @@ class DataStreamTest extends StreamingMultipleProgramsTestBase {
dataStream.print()
val operator = getOperatorForDataStream(dataStream)
.asInstanceOf[AbstractUdfStreamOperator[_, _]]
- return operator.getUserFunction.asInstanceOf[Function]
+ operator.getUserFunction.asInstanceOf[Function]
}
private def getOperatorForDataStream(dataStream: DataStream[_]): StreamOperator[_] = {
@@ -542,15 +523,15 @@ class DataStreamTest extends StreamingMultipleProgramsTestBase {
}
private def isPartitioned(edge: StreamEdge): Boolean = {
- return edge.getPartitioner.isInstanceOf[HashPartitioner[_]]
+ edge.getPartitioner.isInstanceOf[HashPartitioner[_]]
}
private def isCustomPartitioned(edge: StreamEdge): Boolean = {
- return edge.getPartitioner.isInstanceOf[CustomPartitionerWrapper[_, _]]
+ edge.getPartitioner.isInstanceOf[CustomPartitionerWrapper[_, _]]
}
private def createDownStreamId(dataStream: DataStream[_]): Integer = {
- return dataStream.print.getTransformation.getId
+ dataStream.print().getTransformation.getId
}
private def createDownStreamId(dataStream: ConnectedStreams[_, _]): Integer = {
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StateTestPrograms.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StateTestPrograms.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StateTestPrograms.scala
index 650fd7e..7904bcb 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StateTestPrograms.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StateTestPrograms.scala
@@ -17,8 +17,9 @@
*/
package org.apache.flink.streaming.api.scala
+import java.util
+
import org.apache.flink.streaming.api.functions.sink.RichSinkFunction
-import java.util.HashSet
/**
* Test programs for stateful functions.
@@ -30,11 +31,13 @@ object StateTestPrograms {
// test stateful map
env.generateSequence(0, 10).setParallelism(1)
+ .keyBy(x => x)
.mapWithState((in, count: Option[Long]) =>
count match {
- case Some(c) => ((in - c), Some(c + 1))
+ case Some(c) => (in - c, Some(c + 1))
case None => (in, Some(1L))
}).setParallelism(1)
+
.addSink(new RichSinkFunction[Long]() {
var allZero = true
override def invoke(in: Long) = {
@@ -46,13 +49,17 @@ object StateTestPrograms {
})
// test stateful flatmap
- env.fromElements("Fir st-", "Hello world").flatMapWithState((w, s: Option[String]) =>
- s match {
- case Some(s) => (w.split(" ").toList.map(s + _), Some(w))
- case None => (List(w), Some(w))
- }).setParallelism(1)
+ env.fromElements("Fir st-", "Hello world")
+ .keyBy(x => x)
+ .flatMapWithState((w, s: Option[String]) =>
+ s match {
+ case Some(state) => (w.split(" ").toList.map(state + _), Some(w))
+ case None => (List(w), Some(w))
+ })
+ .setParallelism(1)
+
.addSink(new RichSinkFunction[String]() {
- val received = new HashSet[String]()
+ val received = new util.HashSet[String]()
override def invoke(in: String) = { received.add(in) }
override def close() = {
assert(received.size() == 3)
[05/24] flink git commit: [FLINK-2808] [streaming] Refactor and
extend state backend abstraction
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskStateList.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskStateList.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskStateList.java
new file mode 100644
index 0000000..7b8dbd5
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskStateList.java
@@ -0,0 +1,60 @@
+/*
+ * 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.runtime.tasks;
+
+import org.apache.flink.runtime.state.StateHandle;
+
+/**
+ * List of task states for a chain of streaming tasks.
+ */
+public class StreamTaskStateList implements StateHandle<StreamTaskState[]> {
+
+ private static final long serialVersionUID = 1L;
+
+ /** The states for all operator */
+ private final StreamTaskState[] states;
+
+
+ public StreamTaskStateList(StreamTaskState[] states) {
+ this.states = states;
+ }
+
+ public boolean isEmpty() {
+ for (StreamTaskState state : states) {
+ if (state != null) {
+ return false;
+ }
+ }
+ return true;
+ }
+
+ @Override
+ public StreamTaskState[] getState(ClassLoader userCodeClassLoader) {
+ return states;
+ }
+
+ @Override
+ public void discardState() throws Exception {
+ for (StreamTaskState state : states) {
+ if (state != null) {
+ state.discardState();
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamingRuntimeContext.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamingRuntimeContext.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamingRuntimeContext.java
deleted file mode 100644
index a8c4b49..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamingRuntimeContext.java
+++ /dev/null
@@ -1,204 +0,0 @@
-/*
- * 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.runtime.tasks;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.accumulators.Accumulator;
-import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.api.common.functions.util.RuntimeUDFContext;
-import org.apache.flink.api.common.state.OperatorState;
-import org.apache.flink.api.common.state.StateCheckpointer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.execution.Environment;
-import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
-import org.apache.flink.runtime.operators.util.TaskConfig;
-import org.apache.flink.runtime.state.StateHandleProvider;
-import org.apache.flink.streaming.api.state.PartitionedStreamOperatorState;
-import org.apache.flink.streaming.api.state.StreamOperatorState;
-import org.apache.flink.streaming.runtime.operators.Triggerable;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-
-/**
- * Implementation of the {@link RuntimeContext}, created by runtime stream UDF
- * operators.
- */
-public class StreamingRuntimeContext extends RuntimeUDFContext {
-
- private final Environment env;
- private final Map<String, StreamOperatorState<?, ?>> states;
- private final List<PartitionedStreamOperatorState<?, ?, ?>> partitionedStates;
- private final KeySelector<?, ?> statePartitioner;
- private final StateHandleProvider<Serializable> provider;
-
- /**
- * We need access to the {@link StreamTask} to register timer callbacks.
- */
- private final StreamTask<?, ?> streamTask;
-
- @SuppressWarnings("unchecked")
- public StreamingRuntimeContext(
- Environment env,
- ExecutionConfig executionConfig,
- KeySelector<?, ?> statePartitioner,
- StateHandleProvider<?> provider,
- Map<String, Accumulator<?, ?>> accumulatorMap,
- StreamTask<?, ?> streamTask) {
-
- super(env.getTaskName(), env.getNumberOfSubtasks(), env.getIndexInSubtaskGroup(),
- env.getUserClassLoader(), executionConfig,
- env.getDistributedCacheEntries(), accumulatorMap);
-
- this.env = env;
- this.statePartitioner = statePartitioner;
- this.states = new HashMap<>();
- this.partitionedStates = new LinkedList<>();
- this.provider = (StateHandleProvider<Serializable>) provider;
- this.streamTask = streamTask;
- }
-
- /**
- * Returns the input split provider associated with the operator.
- *
- * @return The input split provider.
- */
- public InputSplitProvider getInputSplitProvider() {
- return env.getInputSplitProvider();
- }
-
- /**
- * Returns the stub parameters associated with the {@link TaskConfig} of the
- * operator.
- *
- * @return The stub parameters.
- */
- public Configuration getTaskStubParameters() {
- return new TaskConfig(env.getTaskConfiguration()).getStubParameters();
- }
-
- public StateHandleProvider<Serializable> getStateHandleProvider() {
- return provider;
- }
-
- @SuppressWarnings("unchecked")
- @Override
- public <S, C extends Serializable> OperatorState<S> getOperatorState(String name,
- S defaultState, boolean partitioned, StateCheckpointer<S, C> checkpointer) throws IOException {
- if (defaultState == null) {
- throw new RuntimeException("Cannot set default state to null.");
- }
- StreamOperatorState<S, C> state = (StreamOperatorState<S, C>) getState(name, partitioned);
- state.setCheckpointer(checkpointer);
- state.setDefaultState(defaultState);
-
- return (OperatorState<S>) state;
- }
-
- @SuppressWarnings("unchecked")
- @Override
- public <S extends Serializable> OperatorState<S> getOperatorState(String name, S defaultState,
- boolean partitioned) throws IOException {
- if (defaultState == null) {
- throw new RuntimeException("Cannot set default state to null.");
- }
- StreamOperatorState<S, S> state = (StreamOperatorState<S, S>) getState(name, partitioned);
- state.setDefaultState(defaultState);
-
- return (OperatorState<S>) state;
- }
-
- public StreamOperatorState<?, ?> getState(String name, boolean partitioned) {
- // Try fetching state from the map
- StreamOperatorState<?, ?> state = states.get(name);
- if (state == null) {
- // If not found, create empty state and add to the map
- state = createRawState(partitioned);
- states.put(name, state);
- // We keep a reference to all partitioned states for registering input
- if (state instanceof PartitionedStreamOperatorState) {
- partitionedStates.add((PartitionedStreamOperatorState<?, ?, ?>) state);
- }
- }
- return state;
- }
-
- /**
- * Creates an empty {@link OperatorState}.
- *
- * @return An empty operator state.
- */
- @SuppressWarnings({"rawtypes", "unchecked"})
- public StreamOperatorState<?, ?> createRawState(boolean partitioned) {
- if (partitioned) {
- if (statePartitioner != null) {
- return new PartitionedStreamOperatorState(provider, statePartitioner, getUserCodeClassLoader());
- } else {
- throw new RuntimeException(
- "Partitioned state can only be used with KeyedStreams.");
- }
- } else {
- return new StreamOperatorState(provider);
- }
- }
-
- /**
- * Provides access to the all the states contained in the context
- *
- * @return All the states for the underlying operator.
- */
- public Map<String, StreamOperatorState<?, ?>> getOperatorStates() {
- return states;
- }
-
- /**
- * Register a timer callback. At the specified time the
- * {@code Triggerable } will be invoked. This call is guaranteed to not happen
- * concurrently with method calls on the operator.
- *
- * @param time The absolute time in milliseconds.
- * @param target The target to be triggered.
- */
- public void registerTimer(long time, Triggerable target) {
- streamTask.registerTimer(time, target);
- }
-
- /**
- * Sets the next input of the underlying operators, used to access
- * partitioned states.
- *
- * @param nextRecord
- * Next input of the operator.
- */
- @SuppressWarnings({"unchecked", "rawtypes"})
- public void setNextInput(StreamRecord<?> nextRecord) {
- if (statePartitioner != null) {
- for (PartitionedStreamOperatorState state : partitionedStates) {
- state.setCurrentInput(nextRecord.getValue());
- }
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java
index 25f1a76..d2d8a2e 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTask.java
@@ -23,6 +23,7 @@ import java.util.List;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
+import org.apache.flink.streaming.api.graph.StreamConfig;
import org.apache.flink.streaming.api.graph.StreamEdge;
import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
import org.apache.flink.streaming.runtime.io.StreamTwoInputProcessor;
@@ -35,6 +36,9 @@ public class TwoInputStreamTask<IN1, IN2, OUT> extends StreamTask<OUT, TwoInputS
@Override
public void init() throws Exception {
+ StreamConfig configuration = getConfiguration();
+ ClassLoader userClassLoader = getUserCodeClassLoader();
+
TypeSerializer<IN1> inputDeserializer1 = configuration.getTypeSerializerIn1(userClassLoader);
TypeSerializer<IN2> inputDeserializer2 = configuration.getTypeSerializerIn2(userClassLoader);
@@ -75,10 +79,13 @@ public class TwoInputStreamTask<IN1, IN2, OUT> extends StreamTask<OUT, TwoInputS
@Override
protected void run() throws Exception {
- while (running && inputProcessor.processInput(streamOperator, lock)) {
- if (timerException != null) {
- throw timerException;
- }
+ // cache some references on the stack, to make the code more JIT friendly
+ final TwoInputStreamOperator<IN1, IN2, OUT> operator = this.headOperator;
+ final StreamTwoInputProcessor<IN1, IN2> inputProcessor = this.inputProcessor;
+ final Object lock = getCheckpointLock();
+
+ while (running && inputProcessor.processInput(operator, lock)) {
+ checkTimerException();
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.java
index e002780..aeb5078 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.java
@@ -74,8 +74,7 @@ public class AggregationFunctionTest {
}
// some necessary boiler plate
- TypeInformation<Tuple2<Integer, Integer>> typeInfo = TypeExtractor
- .getForObject(new Tuple2<>(0, 0));
+ TypeInformation<Tuple2<Integer, Integer>> typeInfo = TypeExtractor.getForObject(new Tuple2<>(0, 0));
ExecutionConfig config = new ExecutionConfig();
@@ -92,15 +91,15 @@ public class AggregationFunctionTest {
1, typeInfo, AggregationType.MAX, config);
List<Tuple2<Integer, Integer>> groupedSumList = MockContext.createAndExecute(
- new StreamGroupedReduce<>(sumFunction, keySelector, typeInfo),
+ new StreamGroupedReduce<>(sumFunction, typeInfo.createSerializer(config)),
getInputList());
List<Tuple2<Integer, Integer>> groupedMinList = MockContext.createAndExecute(
- new StreamGroupedReduce<>(minFunction, keySelector, typeInfo),
+ new StreamGroupedReduce<>(minFunction, typeInfo.createSerializer(config)),
getInputList());
List<Tuple2<Integer, Integer>> groupedMaxList = MockContext.createAndExecute(
- new StreamGroupedReduce<>(maxFunction, keySelector, typeInfo),
+ new StreamGroupedReduce<>(maxFunction, typeInfo.createSerializer(config)),
getInputList());
assertEquals(expectedGroupSumList, groupedSumList);
@@ -156,13 +155,13 @@ public class AggregationFunctionTest {
false, config);
List<MyPojo> groupedSumList = MockContext.createAndExecute(
- new StreamGroupedReduce<>(sumFunction, keySelector, typeInfo),
+ new StreamGroupedReduce<>(sumFunction, typeInfo.createSerializer(config)),
getInputPojoList());
List<MyPojo> groupedMinList = MockContext.createAndExecute(
- new StreamGroupedReduce<>(minFunction, keySelector, typeInfo),
+ new StreamGroupedReduce<>(minFunction, typeInfo.createSerializer(config)),
getInputPojoList());
List<MyPojo> groupedMaxList = MockContext.createAndExecute(
- new StreamGroupedReduce<>(maxFunction, keySelector, typeInfo),
+ new StreamGroupedReduce<>(maxFunction, typeInfo.createSerializer(config)),
getInputPojoList());
assertEquals(expectedGroupSumList, groupedSumList);
@@ -216,16 +215,16 @@ public class AggregationFunctionTest {
new ComparableAggregator<>(1, typeInfo, AggregationType.MINBY, false, config);
assertEquals(maxByFirstExpected, MockContext.createAndExecute(
- new StreamGroupedReduce<>(maxByFunctionFirst, keySelector, typeInfo),
+ new StreamGroupedReduce<>(maxByFunctionFirst, typeInfo.createSerializer(config)),
getInputByList()));
assertEquals(maxByLastExpected, MockContext.createAndExecute(
- new StreamGroupedReduce<>(maxByFunctionLast, keySelector, typeInfo),
+ new StreamGroupedReduce<>(maxByFunctionLast, typeInfo.createSerializer(config)),
getInputByList()));
assertEquals(minByLastExpected, MockContext.createAndExecute(
- new StreamGroupedReduce<>(minByFunctionLast, keySelector, typeInfo),
+ new StreamGroupedReduce<>(minByFunctionLast, typeInfo.createSerializer(config)),
getInputByList()));
assertEquals(minByFirstExpected, MockContext.createAndExecute(
- new StreamGroupedReduce<>(minByFunctionFirst, keySelector, typeInfo),
+ new StreamGroupedReduce<>(minByFunctionFirst, typeInfo.createSerializer(config)),
getInputByList()));
}
@@ -274,16 +273,16 @@ public class AggregationFunctionTest {
new ComparableAggregator<>("f1", typeInfo, AggregationType.MINBY, false, config);
assertEquals(maxByFirstExpected, MockContext.createAndExecute(
- new StreamGroupedReduce<>(maxByFunctionFirst, keySelector, typeInfo),
+ new StreamGroupedReduce<>(maxByFunctionFirst, typeInfo.createSerializer(config)),
getInputByPojoList()));
assertEquals(maxByLastExpected, MockContext.createAndExecute(
- new StreamGroupedReduce<>(maxByFunctionLast, keySelector, typeInfo),
+ new StreamGroupedReduce<>(maxByFunctionLast, typeInfo.createSerializer(config)),
getInputByPojoList()));
assertEquals(minByLastExpected, MockContext.createAndExecute(
- new StreamGroupedReduce<>(minByFunctionLast, keySelector, typeInfo),
+ new StreamGroupedReduce<>(minByFunctionLast, typeInfo.createSerializer(config)),
getInputByPojoList()));
assertEquals(minByFirstExpected, MockContext.createAndExecute(
- new StreamGroupedReduce<>(minByFunctionFirst, keySelector, typeInfo),
+ new StreamGroupedReduce<>(minByFunctionFirst, typeInfo.createSerializer(config)),
getInputByPojoList()));
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java
index 3a224e4..c23a4f4 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java
@@ -19,6 +19,7 @@ package org.apache.flink.streaming.api;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@@ -538,7 +539,9 @@ public class DataStreamTest extends StreamingMultipleProgramsTestBase {
DataStreamSink<Long> sink2 = env.generateSequence(1, 100).keyBy(key1).print();
- assertTrue(env.getStreamGraph().getStreamNode(sink2.getTransformation().getId()).getStatePartitioner() != null);
+ assertNotNull(env.getStreamGraph().getStreamNode(sink2.getTransformation().getId()).getStatePartitioner());
+ assertNotNull(env.getStreamGraph().getStreamNode(sink2.getTransformation().getId()).getStateKeySerializer());
+ assertNotNull(env.getStreamGraph().getStreamNode(sink2.getTransformation().getId()).getStateKeySerializer());
assertEquals(key1, env.getStreamGraph().getStreamNode(sink2.getTransformation().getId()).getStatePartitioner());
assertTrue(env.getStreamGraph().getStreamNode(sink2.getTransformation().getId()).getInEdges().get(0).getPartitioner() instanceof HashPartitioner);
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/PrintSinkFunctionTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/PrintSinkFunctionTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/PrintSinkFunctionTest.java
index ac23cda..f7c6e53 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/PrintSinkFunctionTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/functions/PrintSinkFunctionTest.java
@@ -19,7 +19,7 @@ package org.apache.flink.streaming.api.functions;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction;
-import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
import org.junit.After;
import org.junit.Assert;
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java
index 2246ffd..c316604 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/StreamGraphGeneratorTest.java
@@ -35,7 +35,8 @@ import org.apache.flink.streaming.runtime.partitioner.GlobalPartitioner;
import org.apache.flink.streaming.runtime.partitioner.RebalancePartitioner;
import org.apache.flink.streaming.runtime.partitioner.ShufflePartitioner;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
import org.apache.flink.streaming.util.EvenOddOutputSelector;
import org.apache.flink.streaming.util.NoOpIntMap;
import org.apache.flink.streaming.util.NoOpSink;
@@ -262,19 +263,13 @@ public class StreamGraphGeneratorTest extends StreamingMultipleProgramsTestBase
}
@Override
- public void processWatermark1(Watermark mark) throws Exception {
-
- }
+ public void processWatermark1(Watermark mark) throws Exception {}
@Override
- public void processWatermark2(Watermark mark) throws Exception {
-
- }
+ public void processWatermark2(Watermark mark) throws Exception {}
@Override
- public void setup(Output output, StreamingRuntimeContext runtimeContext) {
-
- }
+ public void setup(StreamTask<?, ?> containingTask, StreamConfig config, Output<StreamRecord<Integer>> output) {}
}
private static class OutputTypeConfigurableOperationWithOneInput
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedFoldTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedFoldTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedFoldTest.java
index bc5d614..1002b10 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedFoldTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedFoldTest.java
@@ -43,6 +43,7 @@ import org.junit.Test;
* <li>Watermarks are correctly forwarded</li>
* </ul>
*/
+@SuppressWarnings("serial")
public class StreamGroupedFoldTest {
private static class MyFolder implements FoldFunction<Integer, String> {
@@ -60,20 +61,17 @@ public class StreamGroupedFoldTest {
private TypeInformation<String> outType = TypeExtractor.getForClass(String.class);
@Test
- @SuppressWarnings("unchecked")
public void testGroupedFold() throws Exception {
- StreamGroupedFold<Integer, String> operator = new StreamGroupedFold<>(
- new MyFolder(), new KeySelector<Integer, String>() {
-
- private static final long serialVersionUID = 1L;
-
+ KeySelector<Integer, String> keySelector = new KeySelector<Integer, String>() {
+
@Override
- public String getKey(Integer value) throws Exception {
+ public String getKey(Integer value) {
return value.toString();
}
- }, "100", inType);
-
+ };
+
+ StreamGroupedFold<Integer, String, String> operator = new StreamGroupedFold<>(new MyFolder(), "100");
operator.setOutputType(outType, new ExecutionConfig());
OneInputStreamOperatorTestHarness<Integer, String> testHarness = new OneInputStreamOperatorTestHarness<>(operator);
@@ -102,15 +100,15 @@ public class StreamGroupedFoldTest {
@Test
public void testOpenClose() throws Exception {
- StreamGroupedFold<Integer, String> operator = new StreamGroupedFold<>(new TestOpenCloseFoldFunction(), new KeySelector<Integer, Integer>() {
- private static final long serialVersionUID = 1L;
-
+ KeySelector<Integer, Integer> keySelector = new KeySelector<Integer, Integer>() {
@Override
- public Integer getKey(Integer value) throws Exception {
+ public Integer getKey(Integer value) {
return value;
}
- }, "init", inType);
-
+ };
+
+ StreamGroupedFold<Integer, String, Integer> operator = new StreamGroupedFold<>(
+ new TestOpenCloseFoldFunction(), "init");
operator.setOutputType(BasicTypeInfo.STRING_TYPE_INFO, new ExecutionConfig());
OneInputStreamOperatorTestHarness<Integer, String> testHarness = new OneInputStreamOperatorTestHarness<>(operator);
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedReduceTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedReduceTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedReduceTest.java
index 85d9bc1..b5d2bd6 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedReduceTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedReduceTest.java
@@ -17,14 +17,14 @@
package org.apache.flink.streaming.api.operators;
-
import java.util.concurrent.ConcurrentLinkedQueue;
import org.apache.flink.api.common.functions.ReduceFunction;
import org.apache.flink.api.common.functions.RichReduceFunction;
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
@@ -46,9 +46,11 @@ import org.junit.Test;
public class StreamGroupedReduceTest {
@Test
- @SuppressWarnings("unchecked")
public void testGroupedReduce() throws Exception {
- StreamGroupedReduce<Integer> operator = new StreamGroupedReduce<>(new MyReducer(), new IntegerKeySelector(), typeInfo);
+
+ KeySelector<Integer, Integer> keySelector = new IntegerKeySelector();
+
+ StreamGroupedReduce<Integer> operator = new StreamGroupedReduce<>(new MyReducer(), IntSerializer.INSTANCE);
OneInputStreamOperatorTestHarness<Integer, Integer> testHarness = new OneInputStreamOperatorTestHarness<>(operator);
@@ -76,8 +78,11 @@ public class StreamGroupedReduceTest {
@Test
public void testOpenClose() throws Exception {
+
+ KeySelector<Integer, Integer> keySelector = new IntegerKeySelector();
+
StreamGroupedReduce<Integer> operator =
- new StreamGroupedReduce<>(new TestOpenCloseReduceFunction(), new IntegerKeySelector(), typeInfo);
+ new StreamGroupedReduce<>(new TestOpenCloseReduceFunction(), IntSerializer.INSTANCE);
OneInputStreamOperatorTestHarness<Integer, Integer> testHarness = new OneInputStreamOperatorTestHarness<>(operator);
long initialTime = 0L;
@@ -150,5 +155,5 @@ public class StreamGroupedReduceTest {
}
}
- private static TypeInformation<Integer> typeInfo = TypeExtractor.getForClass(Integer.class);
+ private static TypeInformation<Integer> typeInfo = BasicTypeInfo.INT_TYPE_INFO;
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/FileStateBackendTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/FileStateBackendTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/FileStateBackendTest.java
new file mode 100644
index 0000000..73100d1
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/FileStateBackendTest.java
@@ -0,0 +1,419 @@
+/*
+ * 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.state;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.FloatSerializer;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.api.java.typeutils.runtime.ValueSerializer;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.testutils.CommonTestUtils;
+import org.apache.flink.runtime.state.StateHandle;
+import org.apache.flink.streaming.api.state.filesystem.FileStreamStateHandle;
+import org.apache.flink.streaming.api.state.filesystem.FsStateBackend;
+import org.apache.flink.types.StringValue;
+import org.apache.flink.util.OperatingSystem;
+
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+import java.util.Random;
+import java.util.UUID;
+
+import static org.junit.Assert.*;
+
+public class FileStateBackendTest {
+
+ @Test
+ public void testSetupAndSerialization() {
+ File tempDir = new File(ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH, UUID.randomUUID().toString());
+ try {
+ final String backendDir = localFileUri(tempDir);
+ FsStateBackend originalBackend = new FsStateBackend(backendDir);
+
+ assertFalse(originalBackend.isInitialized());
+ assertEquals(new URI(backendDir), originalBackend.getBasePath().toUri());
+ assertNull(originalBackend.getCheckpointDirectory());
+
+ // serialize / copy the backend
+ FsStateBackend backend = CommonTestUtils.createCopySerializable(originalBackend);
+ assertFalse(backend.isInitialized());
+ assertEquals(new URI(backendDir), backend.getBasePath().toUri());
+ assertNull(backend.getCheckpointDirectory());
+
+ // no file operations should be possible right now
+ try {
+ backend.checkpointStateSerializable("exception train rolling in", 2L, System.currentTimeMillis());
+ fail("should fail with an exception");
+ } catch (IllegalStateException e) {
+ // supreme!
+ }
+
+ backend.initializeForJob(new JobID());
+ assertNotNull(backend.getCheckpointDirectory());
+
+ File checkpointDir = new File(backend.getCheckpointDirectory().toUri().getPath());
+ assertTrue(checkpointDir.exists());
+ assertTrue(isDirectoryEmpty(checkpointDir));
+
+ backend.disposeAllStateForCurrentJob();
+ assertNull(backend.getCheckpointDirectory());
+
+ assertTrue(isDirectoryEmpty(tempDir));
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ finally {
+ deleteDirectorySilently(tempDir);
+ }
+ }
+
+ @Test
+ public void testSerializableState() {
+ File tempDir = new File(ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH, UUID.randomUUID().toString());
+ try {
+ FsStateBackend backend = CommonTestUtils.createCopySerializable(new FsStateBackend(localFileUri(tempDir)));
+ backend.initializeForJob(new JobID());
+
+ File checkpointDir = new File(backend.getCheckpointDirectory().toUri().getPath());
+
+ String state1 = "dummy state";
+ String state2 = "row row row your boat";
+ Integer state3 = 42;
+
+ StateHandle<String> handle1 = backend.checkpointStateSerializable(state1, 439568923746L, System.currentTimeMillis());
+ StateHandle<String> handle2 = backend.checkpointStateSerializable(state2, 439568923746L, System.currentTimeMillis());
+ StateHandle<Integer> handle3 = backend.checkpointStateSerializable(state3, 439568923746L, System.currentTimeMillis());
+
+ assertFalse(isDirectoryEmpty(checkpointDir));
+ assertEquals(state1, handle1.getState(getClass().getClassLoader()));
+ handle1.discardState();
+
+ assertFalse(isDirectoryEmpty(checkpointDir));
+ assertEquals(state2, handle2.getState(getClass().getClassLoader()));
+ handle2.discardState();
+
+ assertFalse(isDirectoryEmpty(checkpointDir));
+ assertEquals(state3, handle3.getState(getClass().getClassLoader()));
+ handle3.discardState();
+
+ assertTrue(isDirectoryEmpty(checkpointDir));
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ finally {
+ deleteDirectorySilently(tempDir);
+ }
+ }
+
+ @Test
+ public void testStateOutputStream() {
+ File tempDir = new File(ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH, UUID.randomUUID().toString());
+ try {
+ FsStateBackend backend = CommonTestUtils.createCopySerializable(new FsStateBackend(localFileUri(tempDir)));
+ backend.initializeForJob(new JobID());
+
+ File checkpointDir = new File(backend.getCheckpointDirectory().toUri().getPath());
+
+ byte[] state1 = new byte[1274673];
+ byte[] state2 = new byte[1];
+ byte[] state3 = new byte[0];
+ byte[] state4 = new byte[177];
+
+ Random rnd = new Random();
+ rnd.nextBytes(state1);
+ rnd.nextBytes(state2);
+ rnd.nextBytes(state3);
+ rnd.nextBytes(state4);
+
+ long checkpointId = 97231523452L;
+
+ FsStateBackend.FsCheckpointStateOutputStream stream1 =
+ backend.createCheckpointStateOutputStream(checkpointId, System.currentTimeMillis());
+ FsStateBackend.FsCheckpointStateOutputStream stream2 =
+ backend.createCheckpointStateOutputStream(checkpointId, System.currentTimeMillis());
+ FsStateBackend.FsCheckpointStateOutputStream stream3 =
+ backend.createCheckpointStateOutputStream(checkpointId, System.currentTimeMillis());
+
+ stream1.write(state1);
+ stream2.write(state2);
+ stream3.write(state3);
+
+ FileStreamStateHandle handle1 = stream1.closeAndGetHandle();
+ FileStreamStateHandle handle2 = stream2.closeAndGetHandle();
+ FileStreamStateHandle handle3 = stream3.closeAndGetHandle();
+
+ // use with try-with-resources
+ StreamStateHandle handle4;
+ try (StateBackend.CheckpointStateOutputStream stream4 =
+ backend.createCheckpointStateOutputStream(checkpointId, System.currentTimeMillis())) {
+ stream4.write(state4);
+ handle4 = stream4.closeAndGetHandle();
+ }
+
+ // close before accessing handle
+ StateBackend.CheckpointStateOutputStream stream5 =
+ backend.createCheckpointStateOutputStream(checkpointId, System.currentTimeMillis());
+ stream5.write(state4);
+ stream5.close();
+ try {
+ stream5.closeAndGetHandle();
+ fail();
+ } catch (IOException e) {
+ // uh-huh
+ }
+
+ validateBytesInStream(handle1.getState(getClass().getClassLoader()), state1);
+ handle1.discardState();
+ assertFalse(isDirectoryEmpty(checkpointDir));
+ ensureLocalFileDeleted(handle1.getFilePath());
+
+ validateBytesInStream(handle2.getState(getClass().getClassLoader()), state2);
+ handle2.discardState();
+ assertFalse(isDirectoryEmpty(checkpointDir));
+ ensureLocalFileDeleted(handle2.getFilePath());
+
+ validateBytesInStream(handle3.getState(getClass().getClassLoader()), state3);
+ handle3.discardState();
+ assertFalse(isDirectoryEmpty(checkpointDir));
+ ensureLocalFileDeleted(handle3.getFilePath());
+
+ validateBytesInStream(handle4.getState(getClass().getClassLoader()), state4);
+ handle4.discardState();
+ assertTrue(isDirectoryEmpty(checkpointDir));
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ finally {
+ deleteDirectorySilently(tempDir);
+ }
+ }
+
+ @Test
+ public void testKeyValueState() {
+ File tempDir = new File(ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH, UUID.randomUUID().toString());
+ try {
+ FsStateBackend backend = CommonTestUtils.createCopySerializable(new FsStateBackend(localFileUri(tempDir)));
+ backend.initializeForJob(new JobID());
+
+ File checkpointDir = new File(backend.getCheckpointDirectory().toUri().getPath());
+
+ KvState<Integer, String, FsStateBackend> kv =
+ backend.createKvState(IntSerializer.INSTANCE, StringSerializer.INSTANCE, null);
+
+ assertEquals(0, kv.size());
+
+ // some modifications to the state
+ kv.setCurrentKey(1);
+ assertNull(kv.value());
+ kv.update("1");
+ assertEquals(1, kv.size());
+ kv.setCurrentKey(2);
+ assertNull(kv.value());
+ kv.update("2");
+ assertEquals(2, kv.size());
+ kv.setCurrentKey(1);
+ assertEquals("1", kv.value());
+ assertEquals(2, kv.size());
+
+ // draw a snapshot
+ KvStateSnapshot<Integer, String, FsStateBackend> snapshot1 =
+ kv.shapshot(682375462378L, System.currentTimeMillis());
+
+ // make some more modifications
+ kv.setCurrentKey(1);
+ kv.update("u1");
+ kv.setCurrentKey(2);
+ kv.update("u2");
+ kv.setCurrentKey(3);
+ kv.update("u3");
+
+ // draw another snapshot
+ KvStateSnapshot<Integer, String, FsStateBackend> snapshot2 =
+ kv.shapshot(682375462379L, System.currentTimeMillis());
+
+ // validate the original state
+ assertEquals(3, kv.size());
+ kv.setCurrentKey(1);
+ assertEquals("u1", kv.value());
+ kv.setCurrentKey(2);
+ assertEquals("u2", kv.value());
+ kv.setCurrentKey(3);
+ assertEquals("u3", kv.value());
+
+ // restore the first snapshot and validate it
+ KvState<Integer, String, FsStateBackend> restored1 = snapshot1.restoreState(backend,
+ IntSerializer.INSTANCE, StringSerializer.INSTANCE, null, getClass().getClassLoader());
+
+ assertEquals(2, restored1.size());
+ restored1.setCurrentKey(1);
+ assertEquals("1", restored1.value());
+ restored1.setCurrentKey(2);
+ assertEquals("2", restored1.value());
+
+ // restore the first snapshot and validate it
+ KvState<Integer, String, FsStateBackend> restored2 = snapshot2.restoreState(backend,
+ IntSerializer.INSTANCE, StringSerializer.INSTANCE, null, getClass().getClassLoader());
+
+ assertEquals(3, restored2.size());
+ restored2.setCurrentKey(1);
+ assertEquals("u1", restored2.value());
+ restored2.setCurrentKey(2);
+ assertEquals("u2", restored2.value());
+ restored2.setCurrentKey(3);
+ assertEquals("u3", restored2.value());
+
+ snapshot1.discardState();
+ assertFalse(isDirectoryEmpty(checkpointDir));
+
+ snapshot2.discardState();
+ assertTrue(isDirectoryEmpty(checkpointDir));
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ finally {
+ deleteDirectorySilently(tempDir);
+ }
+ }
+
+ @Test
+ public void testRestoreWithWrongSerializers() {
+ File tempDir = new File(ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH, UUID.randomUUID().toString());
+ try {
+ FsStateBackend backend = CommonTestUtils.createCopySerializable(new FsStateBackend(localFileUri(tempDir)));
+ backend.initializeForJob(new JobID());
+
+ File checkpointDir = new File(backend.getCheckpointDirectory().toUri().getPath());
+
+ KvState<Integer, String, FsStateBackend> kv =
+ backend.createKvState(IntSerializer.INSTANCE, StringSerializer.INSTANCE, null);
+
+ kv.setCurrentKey(1);
+ kv.update("1");
+ kv.setCurrentKey(2);
+ kv.update("2");
+
+ KvStateSnapshot<Integer, String, FsStateBackend> snapshot =
+ kv.shapshot(682375462378L, System.currentTimeMillis());
+
+
+ @SuppressWarnings("unchecked")
+ TypeSerializer<Integer> fakeIntSerializer =
+ (TypeSerializer<Integer>) (TypeSerializer<?>) FloatSerializer.INSTANCE;
+
+ @SuppressWarnings("unchecked")
+ TypeSerializer<String> fakeStringSerializer =
+ (TypeSerializer<String>) (TypeSerializer<?>) new ValueSerializer<StringValue>(StringValue.class);
+
+ try {
+ snapshot.restoreState(backend, fakeIntSerializer,
+ StringSerializer.INSTANCE, null, getClass().getClassLoader());
+ fail("should recognize wrong serializers");
+ } catch (IllegalArgumentException e) {
+ // expected
+ } catch (Exception e) {
+ fail("wrong exception");
+ }
+
+ try {
+ snapshot.restoreState(backend, IntSerializer.INSTANCE,
+ fakeStringSerializer, null, getClass().getClassLoader());
+ fail("should recognize wrong serializers");
+ } catch (IllegalArgumentException e) {
+ // expected
+ } catch (Exception e) {
+ fail("wrong exception");
+ }
+
+ try {
+ snapshot.restoreState(backend, fakeIntSerializer,
+ fakeStringSerializer, null, getClass().getClassLoader());
+ fail("should recognize wrong serializers");
+ } catch (IllegalArgumentException e) {
+ // expected
+ } catch (Exception e) {
+ fail("wrong exception");
+ }
+
+ snapshot.discardState();
+
+ assertTrue(isDirectoryEmpty(checkpointDir));
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ finally {
+ deleteDirectorySilently(tempDir);
+ }
+ }
+
+ // ------------------------------------------------------------------------
+ // Utilities
+ // ------------------------------------------------------------------------
+
+ private static void ensureLocalFileDeleted(Path path) {
+ URI uri = path.toUri();
+ if ("file".equals(uri.getScheme())) {
+ File file = new File(uri.getPath());
+ assertFalse("file not properly deleted", file.exists());
+ }
+ else {
+ throw new IllegalArgumentException("not a local path");
+ }
+ }
+
+ private static void deleteDirectorySilently(File dir) {
+ try {
+ FileUtils.deleteDirectory(dir);
+ }
+ catch (IOException ignored) {}
+ }
+
+ private static boolean isDirectoryEmpty(File directory) {
+ String[] nested = directory.list();
+ return nested == null || nested.length == 0;
+ }
+
+ private static String localFileUri(File path) {
+ return (OperatingSystem.isWindows() ? "file:/" : "file://") + path.getAbsolutePath();
+ }
+
+ private static void validateBytesInStream(InputStream is, byte[] data) throws IOException {
+ byte[] holder = new byte[data.length];
+ assertEquals("not enough data", holder.length, is.read(holder));
+ assertEquals("too much data", -1, is.read());
+ assertArrayEquals("wrong data", data, holder);
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/MemoryStateBackendTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/MemoryStateBackendTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/MemoryStateBackendTest.java
new file mode 100644
index 0000000..3410d09
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/MemoryStateBackendTest.java
@@ -0,0 +1,278 @@
+/*
+ * 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.state;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.base.FloatSerializer;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
+import org.apache.flink.api.java.typeutils.runtime.ValueSerializer;
+import org.apache.flink.runtime.state.StateHandle;
+import org.apache.flink.streaming.api.state.memory.MemoryStateBackend;
+import org.apache.flink.types.StringValue;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.HashMap;
+
+import static org.junit.Assert.*;
+
+/**
+ * Tests for the {@link org.apache.flink.streaming.api.state.memory.MemoryStateBackend}.
+ */
+public class MemoryStateBackendTest {
+
+ @Test
+ public void testSerializableState() {
+ try {
+ MemoryStateBackend backend = new MemoryStateBackend();
+
+ HashMap<String, Integer> state = new HashMap<>();
+ state.put("hey there", 2);
+ state.put("the crazy brown fox stumbles over a sentence that does not contain every letter", 77);
+
+ StateHandle<HashMap<String, Integer>> handle = backend.checkpointStateSerializable(state, 12, 459);
+ assertNotNull(handle);
+
+ HashMap<String, Integer> restored = handle.getState(getClass().getClassLoader());
+ assertEquals(state, restored);
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testOversizedState() {
+ try {
+ MemoryStateBackend backend = new MemoryStateBackend(10);
+
+ HashMap<String, Integer> state = new HashMap<>();
+ state.put("hey there", 2);
+ state.put("the crazy brown fox stumbles over a sentence that does not contain every letter", 77);
+
+ try {
+ backend.checkpointStateSerializable(state, 12, 459);
+ fail("this should cause an exception");
+ }
+ catch (IOException e) {
+ // now darling, isn't that exactly what we wanted?
+ }
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testStateStream() {
+ try {
+ MemoryStateBackend backend = new MemoryStateBackend();
+
+ HashMap<String, Integer> state = new HashMap<>();
+ state.put("hey there", 2);
+ state.put("the crazy brown fox stumbles over a sentence that does not contain every letter", 77);
+
+ StateBackend.CheckpointStateOutputStream os = backend.createCheckpointStateOutputStream(1, 2);
+ ObjectOutputStream oos = new ObjectOutputStream(os);
+ oos.writeObject(state);
+ oos.flush();
+ StreamStateHandle handle = os.closeAndGetHandle();
+
+ assertNotNull(handle);
+
+ ObjectInputStream ois = new ObjectInputStream(handle.getState(getClass().getClassLoader()));
+ assertEquals(state, ois.readObject());
+ assertTrue(ois.available() <= 0);
+ ois.close();
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testOversizedStateStream() {
+ try {
+ MemoryStateBackend backend = new MemoryStateBackend(10);
+
+ HashMap<String, Integer> state = new HashMap<>();
+ state.put("hey there", 2);
+ state.put("the crazy brown fox stumbles over a sentence that does not contain every letter", 77);
+
+ StateBackend.CheckpointStateOutputStream os = backend.createCheckpointStateOutputStream(1, 2);
+ ObjectOutputStream oos = new ObjectOutputStream(os);
+
+ try {
+ oos.writeObject(state);
+ oos.flush();
+ os.closeAndGetHandle();
+ fail("this should cause an exception");
+ }
+ catch (IOException e) {
+ // oh boy! what an exception!
+ }
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testKeyValueState() {
+ try {
+ MemoryStateBackend backend = new MemoryStateBackend();
+
+ KvState<Integer, String, MemoryStateBackend> kv =
+ backend.createKvState(IntSerializer.INSTANCE, StringSerializer.INSTANCE, null);
+
+ assertEquals(0, kv.size());
+
+ // some modifications to the state
+ kv.setCurrentKey(1);
+ assertNull(kv.value());
+ kv.update("1");
+ assertEquals(1, kv.size());
+ kv.setCurrentKey(2);
+ assertNull(kv.value());
+ kv.update("2");
+ assertEquals(2, kv.size());
+ kv.setCurrentKey(1);
+ assertEquals("1", kv.value());
+ assertEquals(2, kv.size());
+
+ // draw a snapshot
+ KvStateSnapshot<Integer, String, MemoryStateBackend> snapshot1 =
+ kv.shapshot(682375462378L, System.currentTimeMillis());
+
+ // make some more modifications
+ kv.setCurrentKey(1);
+ kv.update("u1");
+ kv.setCurrentKey(2);
+ kv.update("u2");
+ kv.setCurrentKey(3);
+ kv.update("u3");
+
+ // draw another snapshot
+ KvStateSnapshot<Integer, String, MemoryStateBackend> snapshot2 =
+ kv.shapshot(682375462379L, System.currentTimeMillis());
+
+ // validate the original state
+ assertEquals(3, kv.size());
+ kv.setCurrentKey(1);
+ assertEquals("u1", kv.value());
+ kv.setCurrentKey(2);
+ assertEquals("u2", kv.value());
+ kv.setCurrentKey(3);
+ assertEquals("u3", kv.value());
+
+ // restore the first snapshot and validate it
+ KvState<Integer, String, MemoryStateBackend> restored1 = snapshot1.restoreState(backend,
+ IntSerializer.INSTANCE, StringSerializer.INSTANCE, null, getClass().getClassLoader());
+
+ assertEquals(2, restored1.size());
+ restored1.setCurrentKey(1);
+ assertEquals("1", restored1.value());
+ restored1.setCurrentKey(2);
+ assertEquals("2", restored1.value());
+
+ // restore the first snapshot and validate it
+ KvState<Integer, String, MemoryStateBackend> restored2 = snapshot2.restoreState(backend,
+ IntSerializer.INSTANCE, StringSerializer.INSTANCE, null, getClass().getClassLoader());
+
+ assertEquals(3, restored2.size());
+ restored2.setCurrentKey(1);
+ assertEquals("u1", restored2.value());
+ restored2.setCurrentKey(2);
+ assertEquals("u2", restored2.value());
+ restored2.setCurrentKey(3);
+ assertEquals("u3", restored2.value());
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testRestoreWithWrongSerializers() {
+ try {
+ MemoryStateBackend backend = new MemoryStateBackend();
+ KvState<Integer, String, MemoryStateBackend> kv =
+ backend.createKvState(IntSerializer.INSTANCE, StringSerializer.INSTANCE, null);
+
+ kv.setCurrentKey(1);
+ kv.update("1");
+ kv.setCurrentKey(2);
+ kv.update("2");
+
+ KvStateSnapshot<Integer, String, MemoryStateBackend> snapshot =
+ kv.shapshot(682375462378L, System.currentTimeMillis());
+
+
+ @SuppressWarnings("unchecked")
+ TypeSerializer<Integer> fakeIntSerializer =
+ (TypeSerializer<Integer>) (TypeSerializer<?>) FloatSerializer.INSTANCE;
+
+ @SuppressWarnings("unchecked")
+ TypeSerializer<String> fakeStringSerializer =
+ (TypeSerializer<String>) (TypeSerializer<?>) new ValueSerializer<StringValue>(StringValue.class);
+
+ try {
+ snapshot.restoreState(backend, fakeIntSerializer,
+ StringSerializer.INSTANCE, null, getClass().getClassLoader());
+ fail("should recognize wrong serializers");
+ } catch (IllegalArgumentException e) {
+ // expected
+ } catch (Exception e) {
+ fail("wrong exception");
+ }
+
+ try {
+ snapshot.restoreState(backend, IntSerializer.INSTANCE,
+ fakeStringSerializer, null, getClass().getClassLoader());
+ fail("should recognize wrong serializers");
+ } catch (IllegalArgumentException e) {
+ // expected
+ } catch (Exception e) {
+ fail("wrong exception");
+ }
+
+ try {
+ snapshot.restoreState(backend, fakeIntSerializer,
+ fakeStringSerializer, null, getClass().getClassLoader());
+ fail("should recognize wrong serializers");
+ } catch (IllegalArgumentException e) {
+ // expected
+ } catch (Exception e) {
+ fail("wrong exception");
+ }
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StateHandleTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StateHandleTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StateHandleTest.java
deleted file mode 100644
index d6a8a54..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StateHandleTest.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/*
- * 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.state;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.runtime.state.StateHandle;
-import org.apache.flink.util.InstantiationUtil;
-import org.junit.Test;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-
-public class StateHandleTest {
-
- @Test
- public void operatorStateHandleTest() throws Exception {
-
- MockHandle<Serializable> h1 = new MockHandle<Serializable>(1);
-
- OperatorStateHandle opHandle = new OperatorStateHandle(h1, true);
- assertEquals(1, opHandle.getState(this.getClass().getClassLoader()));
-
- OperatorStateHandle dsHandle = serializeDeserialize(opHandle);
- MockHandle<Serializable> h2 = (MockHandle<Serializable>) dsHandle.getHandle();
- assertFalse(h2.discarded);
- assertNotNull(h1.state);
- assertNull(h2.state);
-
- dsHandle.discardState();
-
- assertTrue(h2.discarded);
- }
-
- @Test
- public void wrapperStateHandleTest() throws Exception {
- final ClassLoader cl = this.getClass().getClassLoader();
-
- MockHandle<Serializable> h1 = new MockHandle<Serializable>(1);
- MockHandle<Serializable> h2 = new MockHandle<Serializable>(2);
- StateHandle<Serializable> h3 = new MockHandle<Serializable>(3);
-
- OperatorStateHandle opH1 = new OperatorStateHandle(h1, true);
- OperatorStateHandle opH2 = new OperatorStateHandle(h2, false);
-
- Map<String, OperatorStateHandle> opHandles = ImmutableMap.of("h1", opH1, "h2", opH2);
-
- Tuple2<StateHandle<Serializable>, Map<String, OperatorStateHandle>> fullState = Tuple2.of(h3,
- opHandles);
-
- List<Tuple2<StateHandle<Serializable>, Map<String, OperatorStateHandle>>> chainedStates = ImmutableList
- .of(fullState);
-
- WrapperStateHandle wrapperHandle = new WrapperStateHandle(chainedStates);
-
- WrapperStateHandle dsWrapper = serializeDeserialize(wrapperHandle);
-
- @SuppressWarnings("unchecked")
- Tuple2<StateHandle<Serializable>, Map<String, OperatorStateHandle>> dsFullState = ((List<Tuple2<StateHandle<Serializable>, Map<String, OperatorStateHandle>>>) dsWrapper
- .getState(cl)).get(0);
-
- Map<String, OperatorStateHandle> dsOpHandles = dsFullState.f1;
-
- assertNull(dsFullState.f0.getState(cl));
- assertFalse(((MockHandle<?>) dsFullState.f0).discarded);
- assertFalse(((MockHandle<?>) dsOpHandles.get("h1").getHandle()).discarded);
- assertNull(dsOpHandles.get("h1").getState(cl));
- assertFalse(((MockHandle<?>) dsOpHandles.get("h2").getHandle()).discarded);
- assertNull(dsOpHandles.get("h2").getState(cl));
-
- dsWrapper.discardState();
-
- assertTrue(((MockHandle<?>) dsFullState.f0).discarded);
- assertTrue(((MockHandle<?>) dsOpHandles.get("h1").getHandle()).discarded);
- assertTrue(((MockHandle<?>) dsOpHandles.get("h2").getHandle()).discarded);
-
- }
-
- @SuppressWarnings("unchecked")
- private <X extends StateHandle<?>> X serializeDeserialize(X handle) throws IOException,
- ClassNotFoundException {
- byte[] serialized = InstantiationUtil.serializeObject(handle);
- return (X) InstantiationUtil.deserializeObject(serialized, Thread.currentThread()
- .getContextClassLoader());
- }
-
- @SuppressWarnings("serial")
- private static class MockHandle<T> implements StateHandle<T> {
-
- boolean discarded = false;
- transient T state;
-
- public MockHandle(T state) {
- this.state = state;
- }
-
- @Override
- public void discardState() {
- state = null;
- discarded = true;
- }
-
- @Override
- public T getState(ClassLoader userCodeClassLoader) {
- return state;
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StatefulOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StatefulOperatorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StatefulOperatorTest.java
deleted file mode 100644
index ead3af8..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/state/StatefulOperatorTest.java
+++ /dev/null
@@ -1,377 +0,0 @@
-/*
- * 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.state;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.commons.lang.mutable.MutableInt;
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.accumulators.Accumulator;
-import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.common.state.OperatorState;
-import org.apache.flink.api.common.state.StateCheckpointer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.operators.testutils.MockEnvironment;
-import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;
-import org.apache.flink.runtime.state.LocalStateHandle.LocalStateHandleProvider;
-import org.apache.flink.runtime.state.StateHandle;
-import org.apache.flink.streaming.api.checkpoint.Checkpointed;
-import org.apache.flink.streaming.api.datastream.KeyedStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.sink.SinkFunction;
-import org.apache.flink.streaming.api.operators.Output;
-import org.apache.flink.streaming.api.operators.StreamMap;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.apache.flink.util.InstantiationUtil;
-import org.junit.Test;
-
-import com.google.common.collect.ImmutableMap;
-
-/**
- * Test the functionality supported by stateful user functions for both
- * partitioned and non-partitioned user states. This test mimics the runtime
- * behavior of stateful stream operators.
- */
-public class StatefulOperatorTest extends StreamingMultipleProgramsTestBase {
-
- @Test
- public void simpleStateTest() throws Exception {
-
- List<String> out = new ArrayList<String>();
-
- StreamMap<Integer, String> map = createOperatorWithContext(out, new ModKey(2), null);
- StreamingRuntimeContext context = map.getRuntimeContext();
-
- processInputs(map, Arrays.asList(1, 2, 3, 4, 5));
-
- assertEquals(Arrays.asList("1", "2", "3", "4", "5"), out);
- assertEquals((Integer) 5, context.getOperatorState("counter", 0, false).value());
- assertEquals(ImmutableMap.of(0, new MutableInt(2), 1, new MutableInt(3)), context.getOperatorStates().get("groupCounter").getPartitionedState());
- assertEquals("12345", context.getOperatorState("concat", "", false).value());
- assertEquals((Integer) 5, ((StatefulMapper) map.getUserFunction()).checkpointedCounter);
-
- byte[] serializedState0 = InstantiationUtil.serializeObject(map.getStateSnapshotFromFunction(1, 1));
- // Restore state but snapshot again before calling the value
- byte[] serializedState = InstantiationUtil.serializeObject(createOperatorWithContext(out,
- new ModKey(2), serializedState0).getStateSnapshotFromFunction(1, 1));
-
- StreamMap<Integer, String> restoredMap = createOperatorWithContext(out, new ModKey(2), serializedState);
- StreamingRuntimeContext restoredContext = restoredMap.getRuntimeContext();
-
- assertEquals((Integer) 5, restoredContext.getOperatorState("counter", 0, false).value());
- assertEquals(ImmutableMap.of(0, new MutableInt(2), 1, new MutableInt(3)), context.getOperatorStates().get("groupCounter").getPartitionedState());
- assertEquals("12345", restoredContext.getOperatorState("concat", "", false).value());
- assertEquals((Integer) 5, ((StatefulMapper) restoredMap.getUserFunction()).checkpointedCounter);
- out.clear();
-
- processInputs(restoredMap, Arrays.asList(7, 8));
-
- assertEquals(Arrays.asList("7", "8"), out);
- assertEquals((Integer) 7, restoredContext.getOperatorState("counter", 0, false).value());
- assertEquals(ImmutableMap.of(0, new MutableInt(3), 1, new MutableInt(4)), restoredContext.getOperatorStates().get("groupCounter")
- .getPartitionedState());
- assertEquals("1234578", restoredContext.getOperatorState("concat", "", false).value());
- assertEquals((Integer) 7, ((StatefulMapper) restoredMap.getUserFunction()).checkpointedCounter);
-
- }
-
- @Test
- public void apiTest() throws Exception {
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.setParallelism(3);
-
- KeyedStream<Integer, Integer> keyedStream = env
- .fromCollection(Arrays.asList(0, 1, 2, 3, 4, 5, 6))
- .keyBy(new ModKey(4));
-
- keyedStream.map(new StatefulMapper()).addSink(new SinkFunction<String>() {
- private static final long serialVersionUID = 1L;
-
- public void invoke(String value) throws Exception {
- }
- });
-
- keyedStream.map(new StatefulMapper2()).setParallelism(1).addSink(new SinkFunction<String>() {
- private static final long serialVersionUID = 1L;
-
- public void invoke(String value) throws Exception {
- }
- });
-
- try {
- keyedStream.shuffle();
- fail();
- } catch (UnsupportedOperationException e) {
-
- }
-
- env.fromElements(0, 1, 2, 2, 2, 3, 4, 3, 4).keyBy(new KeySelector<Integer, Integer>() {
- private static final long serialVersionUID = 1L;
-
- @Override
- public Integer getKey(Integer value) throws Exception {
- return value;
- }
-
- }).map(new PStateKeyRemovalTestMapper()).setParallelism(1).addSink(new SinkFunction<String>() {
- private static final long serialVersionUID = 1L;
-
- public void invoke(String value) throws Exception {
- }
- });
-
- env.execute();
- }
-
- private void processInputs(StreamMap<Integer, ?> map, List<Integer> input) throws Exception {
- for (Integer i : input) {
- map.getRuntimeContext().setNextInput(new StreamRecord<Integer>(i, 0L));
- map.processElement(new StreamRecord<Integer>(i, 0L));
- }
- }
-
- @SuppressWarnings("unchecked")
- private StreamMap<Integer, String> createOperatorWithContext(List<String> output,
- KeySelector<Integer, ? extends Serializable> partitioner, byte[] serializedState) throws Exception {
- final List<String> outputList = output;
-
- StreamingRuntimeContext context = new StreamingRuntimeContext(
- new MockEnvironment("MockTask", 3 * 1024 * 1024, new MockInputSplitProvider(), 1024),
- new ExecutionConfig(),
- partitioner,
- new LocalStateHandleProvider<Serializable>(),
- new HashMap<String, Accumulator<?, ?>>(),
- null);
-
- StreamMap<Integer, String> op = new StreamMap<Integer, String>(new StatefulMapper());
-
- op.setup(new Output<StreamRecord<String>>() {
-
- @Override
- public void collect(StreamRecord<String> record) {
- outputList.add(record.getValue());
- }
-
- @Override
- public void emitWatermark(Watermark mark) {
-
- }
-
- @Override
- public void close() {
- }
- }, context);
-
- if (serializedState != null) {
- ClassLoader cl = Thread.currentThread().getContextClassLoader();
- op.restoreInitialState((Tuple2<StateHandle<Serializable>, Map<String, OperatorStateHandle>>) InstantiationUtil
- .deserializeObject(serializedState, cl));
- }
-
- op.open(null);
-
- return op;
- }
-
- public static class StatefulMapper extends RichMapFunction<Integer, String> implements
- Checkpointed<Integer> {
- private static final long serialVersionUID = -9007873655253339356L;
- OperatorState<Integer> counter;
- OperatorState<MutableInt> groupCounter;
- OperatorState<String> concat;
-
- Integer checkpointedCounter = 0;
-
- @Override
- public String map(Integer value) throws Exception {
- counter.update(counter.value() + 1);
- MutableInt incremented = groupCounter.value();
- incremented.increment();
- groupCounter.update(incremented);
- concat.update(concat.value() + value.toString());
- checkpointedCounter++;
- try {
- counter.update(null);
- fail();
- } catch (RuntimeException e) {
- }
- return value.toString();
- }
-
- @Override
- public void open(Configuration conf) throws IOException {
- counter = getRuntimeContext().getOperatorState("counter", 0, false, intCheckpointer);
- groupCounter = getRuntimeContext().getOperatorState("groupCounter", new MutableInt(0), true);
- concat = getRuntimeContext().getOperatorState("concat", "", false);
- try {
- getRuntimeContext().getOperatorState("test", null, true);
- fail();
- } catch (RuntimeException e) {
- }
- try {
- getRuntimeContext().getOperatorState("test", null, true, null);
- fail();
- } catch (RuntimeException e) {
- }
- }
-
- @SuppressWarnings("unchecked")
- @Override
- public void close() throws Exception {
- Map<String, StreamOperatorState<?, ?>> states = ((StreamingRuntimeContext) getRuntimeContext()).getOperatorStates();
- PartitionedStreamOperatorState<Integer, Integer, Integer> groupCounter = (PartitionedStreamOperatorState<Integer, Integer, Integer>) states.get("groupCounter");
- for (Entry<Serializable, Integer> count : groupCounter.getPartitionedState().entrySet()) {
- Integer key = (Integer) count.getKey();
- Integer expected = key < 3 ? 2 : 1;
-
- assertEquals(new MutableInt(expected), count.getValue());
- }
- }
-
- @Override
- public Integer snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
- return checkpointedCounter;
- }
-
- @Override
- public void restoreState(Integer state) {
- this.checkpointedCounter = (Integer) state;
- }
- }
-
- public static class StatefulMapper2 extends RichMapFunction<Integer, String> {
- private static final long serialVersionUID = 1L;
- OperatorState<Integer> groupCounter;
-
- @Override
- public String map(Integer value) throws Exception {
- groupCounter.update(groupCounter.value() + 1);
-
- return value.toString();
- }
-
- @Override
- public void open(Configuration conf) throws IOException {
- groupCounter = getRuntimeContext().getOperatorState("groupCounter", 0, true, intCheckpointer);
- }
-
- @SuppressWarnings("unchecked")
- @Override
- public void close() throws Exception {
- Map<String, StreamOperatorState<?, ?>> states = ((StreamingRuntimeContext) getRuntimeContext()).getOperatorStates();
- PartitionedStreamOperatorState<Integer, Integer, Integer> groupCounter =
- (PartitionedStreamOperatorState<Integer, Integer, Integer>) states.get("groupCounter");
- for (Entry<Serializable, Integer> count : groupCounter.getPartitionedState().entrySet()) {
- Integer key = (Integer) count.getKey();
- Integer expected = key < 3 ? 2 : 1;
- assertEquals(expected, count.getValue());
- }
- }
-
- }
-
- public static StateCheckpointer<Integer, String> intCheckpointer = new StateCheckpointer<Integer, String>() {
-
- private static final long serialVersionUID = 1L;
-
- @Override
- public String snapshotState(Integer state, long checkpointId, long checkpointTimestamp) {
- return state.toString();
- }
-
- @Override
- public Integer restoreState(String stateSnapshot) {
- return Integer.parseInt(stateSnapshot);
- }
- };
-
- public static class PStateKeyRemovalTestMapper extends RichMapFunction<Integer, String> {
-
- private static final long serialVersionUID = 1L;
- OperatorState<Boolean> seen;
-
- @Override
- public String map(Integer value) throws Exception {
- if (value == 0) {
- seen.update(null);
- }else{
- Boolean s = seen.value();
- if (s) {
- seen.update(null);
- } else {
- seen.update(true);
- }
- }
-
- return value.toString();
- }
-
- public void open(Configuration c) throws IOException {
- seen = getRuntimeContext().getOperatorState("seen", false, true);
- }
-
- @SuppressWarnings("unchecked")
- @Override
- public void close() throws Exception {
- Map<String, StreamOperatorState<?, ?>> states = ((StreamingRuntimeContext) getRuntimeContext()).getOperatorStates();
- PartitionedStreamOperatorState<Integer, Boolean, Boolean> seen = (PartitionedStreamOperatorState<Integer, Boolean, Boolean>) states.get("seen");
- assertFalse(seen.getPartitionedState().containsKey(0));
- assertEquals(2,seen.getPartitionedState().size());
- for (Entry<Serializable, Boolean> s : seen.getPartitionedState().entrySet()) {
- assertTrue(s.getValue());
- }
- }
-
- }
-
- public static class ModKey implements KeySelector<Integer, Integer> {
-
- private static final long serialVersionUID = 4193026742083046736L;
-
- int base;
-
- public ModKey(int base) {
- this.base = base;
- }
-
- @Override
- public Integer getKey(Integer value) throws Exception {
- return value % base;
- }
-
- }
-
-}
[02/24] flink git commit: [FLINK-2550] [streaming] Rework JoinStreams
and CoGroupStreams to properly implement operator builder syntax
Posted by se...@apache.org.
[FLINK-2550] [streaming] Rework JoinStreams and CoGroupStreams to properly implement operator builder syntax
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/69dfc40d
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/69dfc40d
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/69dfc40d
Branch: refs/heads/master
Commit: 69dfc40d4b2c9f994d0f828d5f26ed27faaeade0
Parents: c24dca5
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Oct 9 18:20:02 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 16 15:26:10 2015 +0200
----------------------------------------------------------------------
.../api/datastream/CoGroupedStreams.java | 176 ++++++++++---------
.../streaming/api/datastream/DataStream.java | 10 +-
.../streaming/api/datastream/JoinedStreams.java | 158 +++++++++--------
.../streaming/api/scala/CoGroupedStreams.scala | 2 +-
.../streaming/api/scala/JoinedStreams.scala | 4 +-
5 files changed, 182 insertions(+), 168 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/69dfc40d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java
index e1f1a96..d1da783 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java
@@ -17,7 +17,6 @@
package org.apache.flink.streaming.api.datastream;
-import com.google.common.collect.Lists;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.functions.CoGroupFunction;
import org.apache.flink.api.common.functions.MapFunction;
@@ -36,8 +35,11 @@ import org.apache.flink.streaming.api.windowing.windows.Window;
import org.apache.flink.util.Collector;
import java.io.IOException;
+import java.util.ArrayList;
import java.util.List;
+import static java.util.Objects.requireNonNull;
+
/**
*{@code CoGroupedStreams} represents two {@link DataStream DataStreams} that have been co-grouped.
* A streaming co-group operation is evaluated over elements in a window.
@@ -64,93 +66,87 @@ import java.util.List;
* .apply(new MyCoGroupFunction());
* } </pre>
*/
-public class CoGroupedStreams {
+public class CoGroupedStreams<T1, T2> {
- /**
- * A co-group operation that does not yet have its {@link KeySelector KeySelectors} defined.
- *
- * @param <T1> Type of the elements from the first input
- * @param <T2> Type of the elements from the second input
- */
- public static class Unspecified<T1, T2> {
- DataStream<T1> input1;
- DataStream<T2> input2;
+ /** The first input stream */
+ private final DataStream<T1> input1;
- protected Unspecified(DataStream<T1> input1,
- DataStream<T2> input2) {
- this.input1 = input1;
- this.input2 = input2;
- }
+ /** The second input stream */
+ private final DataStream<T2> input2;
- /**
- * Specifies a {@link KeySelector} for elements from the first input.
- */
- public <KEY> WithKey<T1, T2, KEY> where(KeySelector<T1, KEY> keySelector) {
- return new WithKey<>(input1, input2, input1.clean(keySelector), null);
- }
-
- /**
- * Specifies a {@link KeySelector} for elements from the second input.
- */
- public <KEY> WithKey<T1, T2, KEY> equalTo(KeySelector<T2, KEY> keySelector) {
- return new WithKey<>(input1, input2, null, input1.clean(keySelector));
- }
+ /**
+ * Creates new CoGroped data streams, which are the first step towards building a streaming co-group.
+ *
+ * @param input1 The first data stream.
+ * @param input2 The second data stream.
+ */
+ public CoGroupedStreams(DataStream<T1> input1, DataStream<T2> input2) {
+ this.input1 = requireNonNull(input1);
+ this.input2 = requireNonNull(input2);
}
/**
- * A co-group operation that has {@link KeySelector KeySelectors} defined for either both or
- * one input.
- *
- * <p>
- * You need to specify a {@code KeySelector} for both inputs using {@link #where(KeySelector)}
- * and {@link #equalTo(KeySelector)} before you can proceeed with specifying a
- * {@link WindowAssigner} using {@link #window(WindowAssigner)}.
- *
- * @param <T1> Type of the elements from the first input
- * @param <T2> Type of the elements from the second input
- * @param <KEY> Type of the key. This must be the same for both inputs
+ * Specifies a {@link KeySelector} for elements from the first input.
*/
- public static class WithKey<T1, T2, KEY> {
- DataStream<T1> input1;
- DataStream<T2> input2;
+ public <KEY> Where<KEY> where(KeySelector<T1, KEY> keySelector) {
+ TypeInformation<KEY> keyType = TypeExtractor.getKeySelectorTypes(keySelector, input1.getType());
+ return new Where<>(input1.clean(keySelector), keyType);
+ }
- KeySelector<T1, KEY> keySelector1;
- KeySelector<T2, KEY> keySelector2;
+ // ------------------------------------------------------------------------
+
+ /**
+ * CoGrouped streams that have the key for one side defined.
+ *
+ * @param <KEY> The type of the key.
+ */
+ public class Where<KEY> {
- protected WithKey(DataStream<T1> input1, DataStream<T2> input2, KeySelector<T1, KEY> keySelector1, KeySelector<T2, KEY> keySelector2) {
- this.input1 = input1;
- this.input2 = input2;
+ private final KeySelector<T1, KEY> keySelector1;
+ private final TypeInformation<KEY> keyType;
+ Where(KeySelector<T1, KEY> keySelector1, TypeInformation<KEY> keyType) {
this.keySelector1 = keySelector1;
- this.keySelector2 = keySelector2;
- }
-
- /**
- * Specifies a {@link KeySelector} for elements from the first input.
- */
- public WithKey<T1, T2, KEY> where(KeySelector<T1, KEY> keySelector) {
- return new CoGroupedStreams.WithKey<>(input1, input2, input1.clean(keySelector), keySelector2);
+ this.keyType = keyType;
}
-
+
/**
* Specifies a {@link KeySelector} for elements from the second input.
*/
- public CoGroupedStreams.WithKey<T1, T2, KEY> equalTo(KeySelector<T2, KEY> keySelector) {
- return new CoGroupedStreams.WithKey<>(input1, input2, keySelector1, input1.clean(keySelector));
+ public EqualTo equalTo(KeySelector<T2, KEY> keySelector) {
+ TypeInformation<KEY> otherKey = TypeExtractor.getKeySelectorTypes(keySelector, input2.getType());
+ if (!otherKey.equals(this.keyType)) {
+ throw new IllegalArgumentException("The keys for the two inputs are not equal: " +
+ "first key = " + this.keyType + " , second key = " + otherKey);
+ }
+
+ return new EqualTo(input2.clean(keySelector));
}
+ // --------------------------------------------------------------------
+
/**
- * Specifies the window on which the co-group operation works.
+ * A co-group operation that has {@link KeySelector KeySelectors} defined for both inputs.
*/
- public <W extends Window> CoGroupedStreams.WithWindow<T1, T2, KEY, W> window(WindowAssigner<? super TaggedUnion<T1, T2>, W> assigner) {
- if (keySelector1 == null || keySelector2 == null) {
- throw new UnsupportedOperationException("You first need to specify KeySelectors for both inputs using where() and equalTo().");
+ public class EqualTo {
+
+ private final KeySelector<T2, KEY> keySelector2;
+
+ EqualTo(KeySelector<T2, KEY> keySelector2) {
+ this.keySelector2 = requireNonNull(keySelector2);
+ }
+ /**
+ * Specifies the window on which the co-group operation works.
+ */
+ public <W extends Window> WithWindow<T1, T2, KEY, W> window(WindowAssigner<? super TaggedUnion<T1, T2>, W> assigner) {
+ return new WithWindow<>(input1, input2, keySelector1, keySelector2, keyType, assigner, null, null);
}
- return new WithWindow<>(input1, input2, keySelector1, keySelector2, assigner, null, null);
}
}
+ // ------------------------------------------------------------------------
+
/**
* A co-group operation that has {@link KeySelector KeySelectors} defined for both inputs as
* well as a {@link WindowAssigner}.
@@ -166,6 +162,8 @@ public class CoGroupedStreams {
private final KeySelector<T1, KEY> keySelector1;
private final KeySelector<T2, KEY> keySelector2;
+
+ private final TypeInformation<KEY> keyType;
private final WindowAssigner<? super TaggedUnion<T1, T2>, W> windowAssigner;
@@ -177,6 +175,7 @@ public class CoGroupedStreams {
DataStream<T2> input2,
KeySelector<T1, KEY> keySelector1,
KeySelector<T2, KEY> keySelector2,
+ TypeInformation<KEY> keyType,
WindowAssigner<? super TaggedUnion<T1, T2>, W> windowAssigner,
Trigger<? super TaggedUnion<T1, T2>, ? super W> trigger,
Evictor<? super TaggedUnion<T1, T2>, ? super W> evictor) {
@@ -185,7 +184,8 @@ public class CoGroupedStreams {
this.keySelector1 = keySelector1;
this.keySelector2 = keySelector2;
-
+ this.keyType = keyType;
+
this.windowAssigner = windowAssigner;
this.trigger = trigger;
this.evictor = evictor;
@@ -195,7 +195,8 @@ public class CoGroupedStreams {
* Sets the {@code Trigger} that should be used to trigger window emission.
*/
public WithWindow<T1, T2, KEY, W> trigger(Trigger<? super TaggedUnion<T1, T2>, ? super W> newTrigger) {
- return new WithWindow<>(input1, input2, keySelector1, keySelector2, windowAssigner, newTrigger, evictor);
+ return new WithWindow<>(input1, input2, keySelector1, keySelector2, keyType,
+ windowAssigner, newTrigger, evictor);
}
/**
@@ -206,7 +207,8 @@ public class CoGroupedStreams {
* pre-aggregation of window results cannot be used.
*/
public WithWindow<T1, T2, KEY, W> evictor(Evictor<? super TaggedUnion<T1, T2>, ? super W> newEvictor) {
- return new WithWindow<>(input1, input2, keySelector1, keySelector2, windowAssigner, trigger, newEvictor);
+ return new WithWindow<>(input1, input2, keySelector1, keySelector2, keyType,
+ windowAssigner, trigger, newEvictor);
}
/**
@@ -236,16 +238,21 @@ public class CoGroupedStreams {
//clean the closure
function = input1.getExecutionEnvironment().clean(function);
+ UnionTypeInfo<T1, T2> unionType = new UnionTypeInfo<>(input1.getType(), input2.getType());
+ UnionKeySelector<T1, T2, KEY> unionKeySelector = new UnionKeySelector<>(keySelector1, keySelector2);
+
DataStream<TaggedUnion<T1, T2>> taggedInput1 = input1
.map(new Input1Tagger<T1, T2>())
- .returns(new UnionTypeInfo<>(input1.getType(), input2.getType()));
+ .returns(unionType);
DataStream<TaggedUnion<T1, T2>> taggedInput2 = input2
.map(new Input2Tagger<T1, T2>())
- .returns(new UnionTypeInfo<>(input1.getType(), input2.getType()));
+ .returns(unionType);
- WindowedStream<TaggedUnion<T1, T2>, KEY, W> windowOp = taggedInput1
- .union(taggedInput2)
- .keyBy(new UnionKeySelector<>(keySelector1, keySelector2))
+ DataStream<TaggedUnion<T1, T2>> unionStream = taggedInput1.union(taggedInput2);
+
+ // we explicitly create the keyed stream to manually pass the key type information in
+ WindowedStream<TaggedUnion<T1, T2>, KEY, W> windowOp =
+ new KeyedStream<TaggedUnion<T1, T2>, KEY>(unionStream, unionKeySelector, keyType)
.window(windowAssigner);
if (trigger != null) {
@@ -259,13 +266,10 @@ public class CoGroupedStreams {
}
}
- /**
- * Creates a new co-group operation from the two given inputs.
- */
- public static <T1, T2> Unspecified<T1, T2> createCoGroup(DataStream<T1> input1, DataStream<T2> input2) {
- return new Unspecified<>(input1, input2);
- }
-
+ // ------------------------------------------------------------------------
+ // Data type and type information for Tagged Union
+ // ------------------------------------------------------------------------
+
/**
* Internal class for implementing tagged union co-group.
*/
@@ -425,7 +429,7 @@ public class CoGroupedStreams {
@Override
public int getLength() {
- return 0;
+ return -1;
}
@Override
@@ -494,6 +498,11 @@ public class CoGroupedStreams {
}
}
+ // ------------------------------------------------------------------------
+ // Utility functions that implement the CoGroup logic based on the tagged
+ // untion window reduce
+ // ------------------------------------------------------------------------
+
private static class Input1Tagger<T1, T2> implements MapFunction<T1, TaggedUnion<T1, T2>> {
private static final long serialVersionUID = 1L;
@@ -537,6 +546,7 @@ public class CoGroupedStreams {
private static class CoGroupWindowFunction<T1, T2, T, KEY, W extends Window>
extends WrappingFunction<CoGroupFunction<T1, T2, T>>
implements WindowFunction<TaggedUnion<T1, T2>, T, KEY, W> {
+
private static final long serialVersionUID = 1L;
public CoGroupWindowFunction(CoGroupFunction<T1, T2, T> userFunction) {
@@ -548,8 +558,10 @@ public class CoGroupedStreams {
W window,
Iterable<TaggedUnion<T1, T2>> values,
Collector<T> out) throws Exception {
- List<T1> oneValues = Lists.newArrayList();
- List<T2> twoValues = Lists.newArrayList();
+
+ List<T1> oneValues = new ArrayList<>();
+ List<T2> twoValues = new ArrayList<>();
+
for (TaggedUnion<T1, T2> val: values) {
if (val.isOne()) {
oneValues.add(val.getOne());
http://git-wip-us.apache.org/repos/asf/flink/blob/69dfc40d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index 7e686c7..c15ea9b 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -239,7 +239,7 @@ public class DataStream<T> {
* The KeySelector to be used for extracting the key for partitioning
* @return The {@link DataStream} with partitioned state (i.e. KeyedStream)
*/
- public <K> KeyedStream<T, K> keyBy(KeySelector<T, K> key){
+ public <K> KeyedStream<T, K> keyBy(KeySelector<T, K> key) {
return new KeyedStream<T, K>(this, clean(key));
}
@@ -622,16 +622,16 @@ public class DataStream<T> {
* Creates a join operation. See {@link CoGroupedStreams} for an example of how the keys
* and window can be specified.
*/
- public <T2> CoGroupedStreams.Unspecified<T, T2> coGroup(DataStream<T2> otherStream) {
- return CoGroupedStreams.createCoGroup(this, otherStream);
+ public <T2> CoGroupedStreams<T, T2> coGroup(DataStream<T2> otherStream) {
+ return new CoGroupedStreams<>(this, otherStream);
}
/**
* Creates a join operation. See {@link JoinedStreams} for an example of how the keys
* and window can be specified.
*/
- public <T2> JoinedStreams.Unspecified<T, T2> join(DataStream<T2> otherStream) {
- return JoinedStreams.createJoin(this, otherStream);
+ public <T2> JoinedStreams<T, T2> join(DataStream<T2> otherStream) {
+ return new JoinedStreams<>(this, otherStream);
}
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/69dfc40d/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java
index ee848e3..cff9355 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/JoinedStreams.java
@@ -24,12 +24,15 @@ import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.api.java.operators.translation.WrappingFunction;
import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.streaming.api.datastream.CoGroupedStreams.TaggedUnion;
import org.apache.flink.streaming.api.windowing.assigners.WindowAssigner;
import org.apache.flink.streaming.api.windowing.evictors.Evictor;
import org.apache.flink.streaming.api.windowing.triggers.Trigger;
import org.apache.flink.streaming.api.windowing.windows.Window;
import org.apache.flink.util.Collector;
+import static java.util.Objects.requireNonNull;
+
/**
*{@code JoinedStreams} represents two {@link DataStream DataStreams} that have been joined.
* A streaming join operation is evaluated over elements in a window.
@@ -56,92 +59,86 @@ import org.apache.flink.util.Collector;
* .apply(new MyJoinFunction());
* } </pre>
*/
-public class JoinedStreams extends CoGroupedStreams{
+public class JoinedStreams<T1, T2> {
+
+ /** The first input stream */
+ private final DataStream<T1> input1;
+
+ /** The second input stream */
+ private final DataStream<T2> input2;
/**
- * A join operation that does not yet have its {@link KeySelector KeySelectors} defined.
+ * Creates new JoinedStreams data streams, which are the first step towards building a streaming co-group.
*
- * @param <T1> Type of the elements from the first input
- * @param <T2> Type of the elements from the second input
+ * @param input1 The first data stream.
+ * @param input2 The second data stream.
*/
- public static class Unspecified<T1, T2> {
- DataStream<T1> input1;
- DataStream<T2> input2;
-
- protected Unspecified(DataStream<T1> input1,
- DataStream<T2> input2) {
- this.input1 = input1;
- this.input2 = input2;
- }
-
- /**
- * Specifies a {@link KeySelector} for elements from the first input.
- */
- public <KEY> WithKey<T1, T2, KEY> where(KeySelector<T1, KEY> keySelector) {
- return new WithKey<>(input1, input2, keySelector, null);
- }
+ public JoinedStreams(DataStream<T1> input1, DataStream<T2> input2) {
+ this.input1 = requireNonNull(input1);
+ this.input2 = requireNonNull(input2);
+ }
- /**
- * Specifies a {@link KeySelector} for elements from the second input.
- */
- public <KEY> WithKey<T1, T2, KEY> equalTo(KeySelector<T2, KEY> keySelector) {
- return new WithKey<>(input1, input2, null, keySelector);
- }
+ /**
+ * Specifies a {@link KeySelector} for elements from the first input.
+ */
+ public <KEY> Where<KEY> where(KeySelector<T1, KEY> keySelector) {
+ TypeInformation<KEY> keyType = TypeExtractor.getKeySelectorTypes(keySelector, input1.getType());
+ return new Where<>(input1.clean(keySelector), keyType);
}
+ // ------------------------------------------------------------------------
+
/**
- * A join operation that has {@link KeySelector KeySelectors} defined for either both or
- * one input.
- *
- * <p>
- * You need to specify a {@code KeySelector} for both inputs using {@link #where(KeySelector)}
- * and {@link #equalTo(KeySelector)} before you can proceeed with specifying a
- * {@link WindowAssigner} using {@link #window(WindowAssigner)}.
+ * CoGrouped streams that have the key for one side defined.
*
- * @param <T1> Type of the elements from the first input
- * @param <T2> Type of the elements from the second input
- * @param <KEY> Type of the key. This must be the same for both inputs
+ * @param <KEY> The type of the key.
*/
- public static class WithKey<T1, T2, KEY> {
- DataStream<T1> input1;
- DataStream<T2> input2;
-
- KeySelector<T1, KEY> keySelector1;
- KeySelector<T2, KEY> keySelector2;
+ public class Where<KEY> {
- protected WithKey(DataStream<T1> input1, DataStream<T2> input2, KeySelector<T1, KEY> keySelector1, KeySelector<T2, KEY> keySelector2) {
- this.input1 = input1;
- this.input2 = input2;
+ private final KeySelector<T1, KEY> keySelector1;
+ private final TypeInformation<KEY> keyType;
+ Where(KeySelector<T1, KEY> keySelector1, TypeInformation<KEY> keyType) {
this.keySelector1 = keySelector1;
- this.keySelector2 = keySelector2;
- }
-
- /**
- * Specifies a {@link KeySelector} for elements from the first input.
- */
- public WithKey<T1, T2, KEY> where(KeySelector<T1, KEY> keySelector) {
- return new JoinedStreams.WithKey<>(input1, input2, keySelector, keySelector2);
+ this.keyType = keyType;
}
/**
* Specifies a {@link KeySelector} for elements from the second input.
*/
- public JoinedStreams.WithKey<T1, T2, KEY> equalTo(KeySelector<T2, KEY> keySelector) {
- return new JoinedStreams.WithKey<>(input1, input2, keySelector1, keySelector);
+ public EqualTo equalTo(KeySelector<T2, KEY> keySelector) {
+ TypeInformation<KEY> otherKey = TypeExtractor.getKeySelectorTypes(keySelector, input2.getType());
+ if (!otherKey.equals(this.keyType)) {
+ throw new IllegalArgumentException("The keys for the two inputs are not equal: " +
+ "first key = " + this.keyType + " , second key = " + otherKey);
+ }
+
+ return new EqualTo(input2.clean(keySelector));
}
+ // --------------------------------------------------------------------
+
/**
- * Specifies the window on which the join operation works.
+ * A co-group operation that has {@link KeySelector KeySelectors} defined for both inputs.
*/
- public <W extends Window> JoinedStreams.WithWindow<T1, T2, KEY, W> window(WindowAssigner<? super TaggedUnion<T1, T2>, W> assigner) {
- if (keySelector1 == null || keySelector2 == null) {
- throw new UnsupportedOperationException("You first need to specify KeySelectors for both inputs using where() and equalTo().");
+ public class EqualTo {
+ private final KeySelector<T2, KEY> keySelector2;
+
+ EqualTo(KeySelector<T2, KEY> keySelector2) {
+ this.keySelector2 = requireNonNull(keySelector2);
+ }
+
+ /**
+ * Specifies the window on which the co-group operation works.
+ */
+ public <W extends Window> WithWindow<T1, T2, KEY, W> window(WindowAssigner<? super TaggedUnion<T1, T2>, W> assigner) {
+ return new WithWindow<>(input1, input2, keySelector1, keySelector2, keyType, assigner, null, null);
}
- return new WithWindow<>(input1, input2, keySelector1, keySelector2, assigner, null, null);
}
}
+
+ // ------------------------------------------------------------------------
/**
* A join operation that has {@link KeySelector KeySelectors} defined for both inputs as
@@ -153,11 +150,13 @@ public class JoinedStreams extends CoGroupedStreams{
* @param <W> Type of {@link Window} on which the join operation works.
*/
public static class WithWindow<T1, T2, KEY, W extends Window> {
+
private final DataStream<T1> input1;
private final DataStream<T2> input2;
private final KeySelector<T1, KEY> keySelector1;
private final KeySelector<T2, KEY> keySelector2;
+ private final TypeInformation<KEY> keyType;
private final WindowAssigner<? super TaggedUnion<T1, T2>, W> windowAssigner;
@@ -169,16 +168,20 @@ public class JoinedStreams extends CoGroupedStreams{
DataStream<T2> input2,
KeySelector<T1, KEY> keySelector1,
KeySelector<T2, KEY> keySelector2,
+ TypeInformation<KEY> keyType,
WindowAssigner<? super TaggedUnion<T1, T2>, W> windowAssigner,
Trigger<? super TaggedUnion<T1, T2>, ? super W> trigger,
Evictor<? super TaggedUnion<T1, T2>, ? super W> evictor) {
- this.input1 = input1;
- this.input2 = input2;
-
- this.keySelector1 = keySelector1;
- this.keySelector2 = keySelector2;
-
- this.windowAssigner = windowAssigner;
+
+ this.input1 = requireNonNull(input1);
+ this.input2 = requireNonNull(input2);
+
+ this.keySelector1 = requireNonNull(keySelector1);
+ this.keySelector2 = requireNonNull(keySelector2);
+ this.keyType = requireNonNull(keyType);
+
+ this.windowAssigner = requireNonNull(windowAssigner);
+
this.trigger = trigger;
this.evictor = evictor;
}
@@ -187,7 +190,8 @@ public class JoinedStreams extends CoGroupedStreams{
* Sets the {@code Trigger} that should be used to trigger window emission.
*/
public WithWindow<T1, T2, KEY, W> trigger(Trigger<? super TaggedUnion<T1, T2>, ? super W> newTrigger) {
- return new WithWindow<>(input1, input2, keySelector1, keySelector2, windowAssigner, newTrigger, evictor);
+ return new WithWindow<>(input1, input2, keySelector1, keySelector2, keyType,
+ windowAssigner, newTrigger, evictor);
}
/**
@@ -198,7 +202,8 @@ public class JoinedStreams extends CoGroupedStreams{
* pre-aggregation of window results cannot be used.
*/
public WithWindow<T1, T2, KEY, W> evictor(Evictor<? super TaggedUnion<T1, T2>, ? super W> newEvictor) {
- return new WithWindow<>(input1, input2, keySelector1, keySelector2, windowAssigner, trigger, newEvictor);
+ return new WithWindow<>(input1, input2, keySelector1, keySelector2, keyType,
+ windowAssigner, trigger, newEvictor);
}
/**
@@ -213,7 +218,7 @@ public class JoinedStreams extends CoGroupedStreams{
true,
input1.getType(),
input2.getType(),
- "CoGroup",
+ "Join",
false);
return apply(function, resultType);
@@ -249,7 +254,7 @@ public class JoinedStreams extends CoGroupedStreams{
true,
input1.getType(),
input2.getType(),
- "CoGroup",
+ "Join",
false);
return apply(function, resultType);
@@ -273,13 +278,10 @@ public class JoinedStreams extends CoGroupedStreams{
}
}
-
- /**
- * Creates a new join operation from the two given inputs.
- */
- public static <T1, T2> Unspecified<T1, T2> createJoin(DataStream<T1> input1, DataStream<T2> input2) {
- return new Unspecified<>(input1, input2);
- }
+
+ // ------------------------------------------------------------------------
+ // Implementation of the functions
+ // ------------------------------------------------------------------------
/**
* CoGroup function that does a nested-loop join to get the join result.
http://git-wip-us.apache.org/repos/asf/flink/blob/69dfc40d/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/CoGroupedStreams.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/CoGroupedStreams.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/CoGroupedStreams.scala
index 0164b92..e676f81 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/CoGroupedStreams.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/CoGroupedStreams.scala
@@ -270,7 +270,7 @@ object CoGroupedStreams {
*/
def apply[T: TypeInformation](function: CoGroupFunction[T1, T2, T]): DataStream[T] = {
- val coGroup = JavaCoGroupedStreams.createCoGroup(input1.getJavaStream, input2.getJavaStream)
+ val coGroup = new JavaCoGroupedStreams[T1, T2](input1.getJavaStream, input2.getJavaStream)
coGroup
.where(keySelector1)
http://git-wip-us.apache.org/repos/asf/flink/blob/69dfc40d/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/JoinedStreams.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/JoinedStreams.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/JoinedStreams.scala
index 2fda32d..c259724 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/JoinedStreams.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/JoinedStreams.scala
@@ -263,7 +263,7 @@ object JoinedStreams {
*/
def apply[T: TypeInformation](function: JoinFunction[T1, T2, T]): DataStream[T] = {
- val join = JavaJoinedStreams.createJoin(input1.getJavaStream, input2.getJavaStream)
+ val join = new JavaJoinedStreams[T1, T2](input1.getJavaStream, input2.getJavaStream)
join
.where(keySelector1)
@@ -280,7 +280,7 @@ object JoinedStreams {
*/
def apply[T: TypeInformation](function: FlatJoinFunction[T1, T2, T]): DataStream[T] = {
- val join = JavaJoinedStreams.createJoin(input1.getJavaStream, input2.getJavaStream)
+ val join = new JavaJoinedStreams[T1, T2](input1.getJavaStream, input2.getJavaStream)
join
.where(keySelector1)
[21/24] flink git commit: [hotfix] [tests] Make StreamTaskTimerTest
more robust
Posted by se...@apache.org.
[hotfix] [tests] Make StreamTaskTimerTest more robust
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/4ee5b4cc
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/4ee5b4cc
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/4ee5b4cc
Branch: refs/heads/master
Commit: 4ee5b4cc15015f5e985372c6bdace4ef87281825
Parents: 3b7fb63
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Oct 9 22:49:45 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 16 15:26:11 2015 +0200
----------------------------------------------------------------------
.../streaming/runtime/operators/StreamTaskTimerTest.java | 9 ++++-----
1 file changed, 4 insertions(+), 5 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/4ee5b4cc/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskTimerTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskTimerTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskTimerTest.java
index 67df3ad..0ddf272 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskTimerTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/StreamTaskTimerTest.java
@@ -48,6 +48,7 @@ public class StreamTaskTimerTest {
@Test
public void testOpenCloseAndTimestamps() throws Exception {
final OneInputStreamTask<String, String> mapTask = new OneInputStreamTask<>();
+
final OneInputStreamTaskTestHarness<String, String> testHarness = new OneInputStreamTaskTestHarness<>(mapTask, BasicTypeInfo.STRING_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO);
StreamConfig streamConfig = testHarness.getStreamConfig();
@@ -103,11 +104,7 @@ public class StreamTaskTimerTest {
mapTask.registerTimer(t3, new ValidatingTriggerable(errorRef, t3, 2));
mapTask.registerTimer(t4, new ValidatingTriggerable(errorRef, t4, 3));
-
- testHarness.endInput();
- testHarness.waitForTaskCompletion();
-
- long deadline = System.currentTimeMillis() + 5000;
+ long deadline = System.currentTimeMillis() + 20000;
while (errorRef.get() == null &&
ValidatingTriggerable.numInSequence < 4 &&
System.currentTimeMillis() < deadline)
@@ -123,6 +120,8 @@ public class StreamTaskTimerTest {
assertEquals(4, ValidatingTriggerable.numInSequence);
+ testHarness.endInput();
+ testHarness.waitForTaskCompletion();
// wait until the trigger thread is shut down. otherwise, the other tests may become unstable
deadline = System.currentTimeMillis() + 4000;
[14/24] flink git commit: [hotfix] Proper exception chaining in
key/value state access of StreamingRuntimeContext
Posted by se...@apache.org.
[hotfix] Proper exception chaining in key/value state access of StreamingRuntimeContext
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/ca8c73db
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/ca8c73db
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/ca8c73db
Branch: refs/heads/master
Commit: ca8c73db2dbb4744968f4c8fa77841f44cb1773b
Parents: b81598e
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Oct 9 19:34:16 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 16 15:26:10 2015 +0200
----------------------------------------------------------------------
.../flink/streaming/api/operators/StreamingRuntimeContext.java | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/ca8c73db/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java
index 40998dd..a51bb27 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java
@@ -117,7 +117,7 @@ public class StreamingRuntimeContext extends AbstractRuntimeUDFContext {
catch (Exception e) {
throw new RuntimeException("Cannot analyze type '" + stateType.getName() +
"' from the class alone, due to generic type parameters. " +
- "Please specify the TypeInformation directly.");
+ "Please specify the TypeInformation directly.", e);
}
return getKeyValueState(typeInfo, defaultState);
[11/24] flink git commit: [hotfix] Remove remaning classes for old
window triggers
Posted by se...@apache.org.
[hotfix] Remove remaning classes for old window triggers
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/5ac2872f
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/5ac2872f
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/5ac2872f
Branch: refs/heads/master
Commit: 5ac2872f4fb39feded8873eea75b1b4bbbd4842f
Parents: eec2d15
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Oct 9 12:53:57 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 16 15:26:10 2015 +0200
----------------------------------------------------------------------
.../streaming/api/state/CircularFifoList.java | 112 ------
.../api/state/NullableCircularBuffer.java | 362 -------------------
2 files changed, 474 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/5ac2872f/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/CircularFifoList.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/CircularFifoList.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/CircularFifoList.java
deleted file mode 100644
index 98c57d3..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/CircularFifoList.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/*
- * 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.state;
-
-import java.io.Serializable;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Queue;
-
-/**
- * A simple class, that manages a circular queue with sliding interval. If the
- * queue if full and a new element is added, the elements that belong to the
- * first sliding interval are removed.
- */
-public class CircularFifoList<T> implements Serializable {
- private static final long serialVersionUID = 1L;
-
- private Queue<T> queue;
- private Queue<Long> slideSizes;
- private long counter;
- private Iterable<T> iterable;
-
- public CircularFifoList() {
- this.queue = new LinkedList<T>();
- this.slideSizes = new LinkedList<Long>();
- this.counter = 0;
- this.iterable = new ListIterable();
- }
-
- public void add(T element) {
- queue.add(element);
- counter++;
- }
-
- public void newSlide() {
- slideSizes.add(counter);
- counter = 0;
- }
-
- public void shiftWindow() {
- shiftWindow(1);
- }
-
- public void shiftWindow(int numberOfSlides) {
-
- if (numberOfSlides <= slideSizes.size()) {
- for (int i = 0; i < numberOfSlides; i++) {
- Long firstSlideSize = slideSizes.remove();
-
- for (int j = 0; j < firstSlideSize; j++) {
- queue.remove();
- }
- }
- } else {
- slideSizes.clear();
- queue.clear();
- counter = 0;
- }
-
- }
-
- @SuppressWarnings("unchecked")
- public List<T> getElements(){
- return (List<T>) queue;
- }
-
- public Iterator<T> getIterator() {
- return queue.iterator();
- }
-
- public Iterable<T> getIterable() {
- return iterable;
- }
-
- private class ListIterable implements Iterable<T>, Serializable {
-
- private static final long serialVersionUID = 1L;
-
- @Override
- public Iterator<T> iterator() {
- return getIterator();
- }
-
- }
-
- public boolean isEmpty() {
- return queue.isEmpty();
- }
-
- @Override
- public String toString() {
- return queue.toString();
- }
-
-
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/5ac2872f/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/NullableCircularBuffer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/NullableCircularBuffer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/NullableCircularBuffer.java
deleted file mode 100644
index 37d8955..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/NullableCircularBuffer.java
+++ /dev/null
@@ -1,362 +0,0 @@
-/*
- * 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.state;
-
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.io.Serializable;
-import java.util.AbstractCollection;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-
-import org.apache.commons.collections.BoundedCollection;
-import org.apache.commons.collections.Buffer;
-import org.apache.commons.collections.BufferUnderflowException;
-
-@SuppressWarnings("rawtypes")
-public class NullableCircularBuffer extends AbstractCollection implements Buffer,
- BoundedCollection, Serializable {
-
- /** Serialization version */
- private static final long serialVersionUID = 5603722811189451017L;
-
- /** Underlying storage array */
- private transient Object[] elements;
-
- /** Array index of first (oldest) buffer element */
- private transient int start = 0;
-
- /**
- * Index mod maxElements of the array position following the last buffer
- * element. Buffer elements start at elements[start] and "wrap around"
- * elements[maxElements-1], ending at elements[decrement(end)]. For example,
- * elements = {c,a,b}, start=1, end=1 corresponds to the buffer [a,b,c].
- */
- private transient int end = 0;
-
- /** Flag to indicate if the buffer is currently full. */
- private transient boolean full = false;
-
- /** Capacity of the buffer */
- private final int maxElements;
-
- /**
- * Constructs a new <code>BoundedFifoBuffer</code> big enough to hold 32
- * elements.
- */
- public NullableCircularBuffer() {
- this(32);
- }
-
- /**
- * Constructs a new <code>BoundedFifoBuffer</code> big enough to hold the
- * specified number of elements.
- *
- * @param size
- * the maximum number of elements for this fifo
- * @throws IllegalArgumentException
- * if the size is less than 1
- */
- public NullableCircularBuffer(int size) {
- if (size <= 0) {
- throw new IllegalArgumentException("The size must be greater than 0");
- }
- elements = new Object[size];
- maxElements = elements.length;
- }
-
- /**
- * Constructs a new <code>BoundedFifoBuffer</code> big enough to hold all of
- * the elements in the specified collection. That collection's elements will
- * also be added to the buffer.
- *
- * @param coll
- * the collection whose elements to add, may not be null
- * @throws NullPointerException
- * if the collection is null
- */
- @SuppressWarnings("unchecked")
- public NullableCircularBuffer(Collection coll) {
- this(coll.size());
- addAll(coll);
- }
-
- // -----------------------------------------------------------------------
- /**
- * Write the buffer out using a custom routine.
- *
- * @param out
- * the output stream
- * @throws IOException
- */
- private void writeObject(ObjectOutputStream out) throws IOException {
- out.defaultWriteObject();
- out.writeInt(size());
- for (Iterator it = iterator(); it.hasNext();) {
- out.writeObject(it.next());
- }
- }
-
- /**
- * Read the buffer in using a custom routine.
- *
- * @param in
- * the input stream
- * @throws IOException
- * @throws ClassNotFoundException
- */
- private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
- in.defaultReadObject();
- elements = new Object[maxElements];
- int size = in.readInt();
- for (int i = 0; i < size; i++) {
- elements[i] = in.readObject();
- }
- start = 0;
- full = (size == maxElements);
- if (full) {
- end = 0;
- } else {
- end = size;
- }
- }
-
- // -----------------------------------------------------------------------
- /**
- * Returns the number of elements stored in the buffer.
- *
- * @return this buffer's size
- */
- public int size() {
- int size = 0;
-
- if (end < start) {
- size = maxElements - start + end;
- } else if (end == start) {
- size = (full ? maxElements : 0);
- } else {
- size = end - start;
- }
-
- return size;
- }
-
- /**
- * Returns true if this buffer is empty; false otherwise.
- *
- * @return true if this buffer is empty
- */
- public boolean isEmpty() {
- return size() == 0;
- }
-
- /**
- * Returns true if this collection is full and no new elements can be added.
- *
- * @return <code>true</code> if the collection is full
- */
- public boolean isFull() {
- return size() == maxElements;
- }
-
- /**
- * Gets the maximum size of the collection (the bound).
- *
- * @return the maximum number of elements the collection can hold
- */
- public int maxSize() {
- return maxElements;
- }
-
- /**
- * Clears this buffer.
- */
- public void clear() {
- full = false;
- start = 0;
- end = 0;
- Arrays.fill(elements, null);
- }
-
- /**
- * Adds the given element to this buffer.
- *
- * @param element
- * the element to add
- * @return true, always
- */
- public boolean add(Object element) {
-
- if (isFull()) {
- remove();
- }
-
- elements[end++] = element;
-
- if (end >= maxElements) {
- end = 0;
- }
-
- if (end == start) {
- full = true;
- }
-
- return true;
- }
-
- /**
- * Returns the least recently inserted element in this buffer.
- *
- * @return the least recently inserted element
- * @throws BufferUnderflowException
- * if the buffer is empty
- */
- public Object get() {
- if (isEmpty()) {
- throw new BufferUnderflowException("The buffer is already empty");
- }
-
- return elements[start];
- }
-
- /**
- * Removes the least recently inserted element from this buffer.
- *
- * @return the least recently inserted element
- * @throws BufferUnderflowException
- * if the buffer is empty
- */
- public Object remove() {
- if (isEmpty()) {
- throw new BufferUnderflowException("The buffer is already empty");
- }
-
- Object element = elements[start];
-
- elements[start++] = null;
-
- if (start >= maxElements) {
- start = 0;
- }
-
- full = false;
-
- return element;
- }
-
- /**
- * Increments the internal index.
- *
- * @param index
- * the index to increment
- * @return the updated index
- */
- private int increment(int index) {
- index++;
- if (index >= maxElements) {
- index = 0;
- }
- return index;
- }
-
- /**
- * Decrements the internal index.
- *
- * @param index
- * the index to decrement
- * @return the updated index
- */
- private int decrement(int index) {
- index--;
- if (index < 0) {
- index = maxElements - 1;
- }
- return index;
- }
-
- /**
- * Returns an iterator over this buffer's elements.
- *
- * @return an iterator over this buffer's elements
- */
- public Iterator iterator() {
- return new Iterator() {
-
- private int index = start;
- private int lastReturnedIndex = -1;
- private boolean isFirst = full;
-
- public boolean hasNext() {
- return isFirst || (index != end);
-
- }
-
- public Object next() {
- if (!hasNext()) {
- throw new NoSuchElementException();
- }
- isFirst = false;
- lastReturnedIndex = index;
- index = increment(index);
- return elements[lastReturnedIndex];
- }
-
- public void remove() {
- if (lastReturnedIndex == -1) {
- throw new IllegalStateException();
- }
-
- // First element can be removed quickly
- if (lastReturnedIndex == start) {
- NullableCircularBuffer.this.remove();
- lastReturnedIndex = -1;
- return;
- }
-
- int pos = lastReturnedIndex + 1;
- if (start < lastReturnedIndex && pos < end) {
- // shift in one part
- System.arraycopy(elements, pos, elements, lastReturnedIndex, end - pos);
- } else {
- // Other elements require us to shift the subsequent
- // elements
- while (pos != end) {
- if (pos >= maxElements) {
- elements[pos - 1] = elements[0];
- pos = 0;
- } else {
- elements[decrement(pos)] = elements[pos];
- pos = increment(pos);
- }
- }
- }
-
- lastReturnedIndex = -1;
- end = decrement(end);
- elements[end] = null;
- full = false;
- index = decrement(index);
- }
-
- };
- }
-
-}
[17/24] flink git commit: [hotfix] [core] TypeExtractor correctly
handles non-public types as generic types (rather than failing with an
exception)
Posted by se...@apache.org.
[hotfix] [core] TypeExtractor correctly handles non-public types as generic types (rather than failing with an exception)
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/b81598ef
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/b81598ef
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/b81598ef
Branch: refs/heads/master
Commit: b81598ef25176c5b712975ef25c9bfe4ef0f16b3
Parents: d5a016c
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Oct 9 19:31:30 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 16 15:26:10 2015 +0200
----------------------------------------------------------------------
.../apache/flink/api/java/typeutils/TypeExtractor.java | 11 ++++++++---
1 file changed, 8 insertions(+), 3 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/b81598ef/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
----------------------------------------------------------------------
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
index 252842e..1dec90b 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
+++ b/flink-java/src/main/java/org/apache/flink/api/java/typeutils/TypeExtractor.java
@@ -1239,7 +1239,7 @@ public class TypeExtractor {
// check for subclasses of Tuple
if (Tuple.class.isAssignableFrom(clazz)) {
if(clazz == Tuple0.class) {
- return new TupleTypeInfo(Tuple0.class, new TypeInformation<?>[0]);
+ return new TupleTypeInfo(Tuple0.class);
}
throw new InvalidTypesException("Type information extraction for tuples (except Tuple0) cannot be done based on the class.");
}
@@ -1352,17 +1352,22 @@ public class TypeExtractor {
protected <OUT, IN1, IN2> TypeInformation<OUT> analyzePojo(Class<OUT> clazz, ArrayList<Type> typeHierarchy,
ParameterizedType parameterizedType, TypeInformation<IN1> in1Type, TypeInformation<IN2> in2Type) {
+ if (!Modifier.isPublic(clazz.getModifiers())) {
+ LOG.info("Class " + clazz.getName() + " is not public, cannot treat it as a POJO type. Will be handled as GenericType");
+ return new GenericTypeInfo<OUT>(clazz);
+ }
+
// add the hierarchy of the POJO itself if it is generic
if (parameterizedType != null) {
getTypeHierarchy(typeHierarchy, parameterizedType, Object.class);
}
// create a type hierarchy, if the incoming only contains the most bottom one or none.
- else if(typeHierarchy.size() <= 1) {
+ else if (typeHierarchy.size() <= 1) {
getTypeHierarchy(typeHierarchy, clazz, Object.class);
}
List<Field> fields = getAllDeclaredFields(clazz);
- if(fields.size() == 0) {
+ if (fields.size() == 0) {
LOG.info("No fields detected for " + clazz + ". Cannot be used as a PojoType. Will be handled as GenericType");
return new GenericTypeInfo<OUT>(clazz);
}
[16/24] flink git commit: [hotfix] [streaming] Initialize
StreamingRuntimeContext to rich functions early
Posted by se...@apache.org.
[hotfix] [streaming] Initialize StreamingRuntimeContext to rich functions early
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/f2d50386
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/f2d50386
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/f2d50386
Branch: refs/heads/master
Commit: f2d50386c8ad3c2faa492cfbe7c7f75dbbc49035
Parents: 69dfc40
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Oct 9 18:20:53 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 16 15:26:10 2015 +0200
----------------------------------------------------------------------
.../api/operators/AbstractUdfStreamOperator.java | 12 +++++++++++-
1 file changed, 11 insertions(+), 1 deletion(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/f2d50386/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java
index 7a1bea4..a991fd3 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java
@@ -26,7 +26,10 @@ import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.state.StateHandle;
import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier;
import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.graph.StreamConfig;
import org.apache.flink.streaming.api.state.StateBackend;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
import org.apache.flink.streaming.runtime.tasks.StreamTaskState;
import static java.util.Objects.requireNonNull;
@@ -69,11 +72,18 @@ public abstract class AbstractUdfStreamOperator<OUT, F extends Function> extends
// operator life cycle
// ------------------------------------------------------------------------
+
+ @Override
+ public void setup(StreamTask<?, ?> containingTask, StreamConfig config, Output<StreamRecord<OUT>> output) {
+ super.setup(containingTask, config, output);
+
+ FunctionUtils.setFunctionRuntimeContext(userFunction, getRuntimeContext());
+ }
+
@Override
public void open() throws Exception {
super.open();
- FunctionUtils.setFunctionRuntimeContext(userFunction, getRuntimeContext());
FunctionUtils.openFunction(userFunction, new Configuration());
}
[18/24] flink git commit: [FLINK-2846] [streaming] Emit downstream
checkpoint barriers at beginning of the checkpoint scope
Posted by se...@apache.org.
[FLINK-2846] [streaming] Emit downstream checkpoint barriers at beginning of the checkpoint scope
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/da159ef5
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/da159ef5
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/da159ef5
Branch: refs/heads/master
Commit: da159ef563e934cda6f802aa5535986356760f35
Parents: ca8c73d
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Oct 9 21:40:28 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 16 15:26:11 2015 +0200
----------------------------------------------------------------------
.../flink/streaming/runtime/tasks/StreamTask.java | 14 +++++++-------
1 file changed, 7 insertions(+), 7 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/da159ef5/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
index b53d9c4..b607433 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
@@ -422,6 +422,13 @@ public abstract class StreamTask<OUT, Operator extends StreamOperator<OUT>>
synchronized (lock) {
if (isRunning) {
+
+ // since both state checkpointing and downstream barrier emission occurs in this
+ // lock scope, they are an atomic operation regardless of the order in which they occur
+ // we immediately emit the checkpoint barriers, so the downstream operators can start
+ // their checkpoint work as soon as possible
+ operatorChain.broadcastCheckpointBarrier(checkpointId, timestamp);
+
// now draw the state snapshot
try {
final StreamOperator<?>[] allOperators = operatorChain.getAllOperators();
@@ -436,13 +443,6 @@ public abstract class StreamTask<OUT, Operator extends StreamOperator<OUT>>
}
StreamTaskStateList allStates = new StreamTaskStateList(states);
-
- // since both state checkpointing and downstream barrier emission occurs in this
- // lock scope, they are an atomic operation regardless of the order in which they occur
- // we immediately emit the checkpoint barriers, so the downstream operators can start
- // their checkpoint work as soon as possible
- operatorChain.broadcastCheckpointBarrier(checkpointId, timestamp);
-
if (allStates.isEmpty()) {
getEnvironment().acknowledgeCheckpoint(checkpointId);
} else {
[12/24] flink git commit: [FLINK-2550] [streaming] Make fast-path
processing time windows fault tolerant
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/c24dca50/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
index 671544e..dd76a67 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingAlignedProcessingTimeWindowOperatorTest.java
@@ -20,23 +20,30 @@ package org.apache.flink.streaming.runtime.operators.windowing;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.accumulators.Accumulator;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.execution.Environment;
import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
import org.apache.flink.streaming.api.graph.StreamConfig;
import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.state.StateBackend;
+import org.apache.flink.streaming.api.state.memory.MemoryStateBackend;
import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
import org.apache.flink.streaming.runtime.operators.Triggerable;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.apache.flink.streaming.runtime.tasks.StreamTaskState;
import org.apache.flink.util.Collector;
import org.junit.After;
import org.junit.Test;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
+import org.mockito.stubbing.OngoingStubbing;
+import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
@@ -119,27 +126,31 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
@Test
public void testWindowSizeAndSlide() {
try {
- AbstractAlignedProcessingTimeWindowOperator<String, String, String, ?> op;
+ AccumulatingProcessingTimeWindowOperator<String, String, String> op;
- op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector, 5000, 1000);
+ op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
+ StringSerializer.INSTANCE, StringSerializer.INSTANCE, 5000, 1000);
assertEquals(5000, op.getWindowSize());
assertEquals(1000, op.getWindowSlide());
assertEquals(1000, op.getPaneSize());
assertEquals(5, op.getNumPanesPerWindow());
- op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector, 1000, 1000);
+ op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
+ StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1000, 1000);
assertEquals(1000, op.getWindowSize());
assertEquals(1000, op.getWindowSlide());
assertEquals(1000, op.getPaneSize());
assertEquals(1, op.getNumPanesPerWindow());
- op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector, 1500, 1000);
+ op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
+ StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1500, 1000);
assertEquals(1500, op.getWindowSize());
assertEquals(1000, op.getWindowSlide());
assertEquals(500, op.getPaneSize());
assertEquals(3, op.getNumPanesPerWindow());
- op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector, 1200, 1100);
+ op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
+ StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1200, 1100);
assertEquals(1200, op.getWindowSize());
assertEquals(1100, op.getWindowSlide());
assertEquals(100, op.getPaneSize());
@@ -157,31 +168,35 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
@SuppressWarnings("unchecked")
final Output<StreamRecord<String>> mockOut = mock(Output.class);
final StreamTask<?, ?> mockTask = createMockTask();
-
- AbstractAlignedProcessingTimeWindowOperator<String, String, String, ?> op;
- op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector, 5000, 1000);
+ AccumulatingProcessingTimeWindowOperator<String, String, String> op;
+
+ op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
+ StringSerializer.INSTANCE, StringSerializer.INSTANCE, 5000, 1000);
op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
op.open();
assertTrue(op.getNextSlideTime() % 1000 == 0);
assertTrue(op.getNextEvaluationTime() % 1000 == 0);
op.dispose();
- op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector, 1000, 1000);
+ op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
+ StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1000, 1000);
op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
op.open();
assertTrue(op.getNextSlideTime() % 1000 == 0);
assertTrue(op.getNextEvaluationTime() % 1000 == 0);
op.dispose();
- op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector, 1500, 1000);
+ op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
+ StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1500, 1000);
op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
op.open();
assertTrue(op.getNextSlideTime() % 500 == 0);
assertTrue(op.getNextEvaluationTime() % 1000 == 0);
op.dispose();
- op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector, 1200, 1100);
+ op = new AccumulatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
+ StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1200, 1100);
op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
op.open();
assertTrue(op.getNextSlideTime() % 100 == 0);
@@ -204,9 +219,11 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
// tumbling window that triggers every 20 milliseconds
- AbstractAlignedProcessingTimeWindowOperator<Integer, Integer, Integer, ?> op =
+ AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
new AccumulatingProcessingTimeWindowOperator<>(
- validatingIdentityFunction, identitySelector, windowSize, windowSize);
+ validatingIdentityFunction, identitySelector,
+ IntSerializer.INSTANCE, IntSerializer.INSTANCE,
+ windowSize, windowSize);
op.setup(mockTask, new StreamConfig(new Configuration()), out);
op.open();
@@ -220,7 +237,9 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
Thread.sleep(1);
}
- op.close();
+ synchronized (lock) {
+ op.close();
+ }
op.dispose();
// get and verify the result
@@ -250,8 +269,10 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
// tumbling window that triggers every 20 milliseconds
- AbstractAlignedProcessingTimeWindowOperator<Integer, Integer, Integer, ?> op =
- new AccumulatingProcessingTimeWindowOperator<>(validatingIdentityFunction, identitySelector, 150, 50);
+ AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
+ new AccumulatingProcessingTimeWindowOperator<>(
+ validatingIdentityFunction, identitySelector,
+ IntSerializer.INSTANCE, IntSerializer.INSTANCE, 150, 50);
op.setup(mockTask, new StreamConfig(new Configuration()), out);
op.open();
@@ -265,7 +286,9 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
Thread.sleep(1);
}
- op.close();
+ synchronized (lock) {
+ op.close();
+ }
op.dispose();
// get and verify the result
@@ -312,8 +335,10 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
// tumbling window that triggers every 20 milliseconds
- AbstractAlignedProcessingTimeWindowOperator<Integer, Integer, Integer, ?> op =
- new AccumulatingProcessingTimeWindowOperator<>(validatingIdentityFunction, identitySelector, 50, 50);
+ AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
+ new AccumulatingProcessingTimeWindowOperator<>(
+ validatingIdentityFunction, identitySelector,
+ IntSerializer.INSTANCE, IntSerializer.INSTANCE, 50, 50);
op.setup(mockTask, new StreamConfig(new Configuration()), out);
op.open();
@@ -342,7 +367,9 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
Collections.sort(result);
assertEquals(Arrays.asList(1, 2, 3, 4, 5, 6), result);
- op.close();
+ synchronized (lock) {
+ op.close();
+ }
op.dispose();
}
catch (Exception e) {
@@ -364,8 +391,10 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
// tumbling window that triggers every 20 milliseconds
- AbstractAlignedProcessingTimeWindowOperator<Integer, Integer, Integer, ?> op =
- new AccumulatingProcessingTimeWindowOperator<>(validatingIdentityFunction, identitySelector, 150, 50);
+ AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
+ new AccumulatingProcessingTimeWindowOperator<>(
+ validatingIdentityFunction, identitySelector,
+ IntSerializer.INSTANCE, IntSerializer.INSTANCE, 150, 50);
op.setup(mockTask, new StreamConfig(new Configuration()), out);
op.open();
@@ -384,8 +413,10 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
Collections.sort(result);
assertEquals(Arrays.asList(1, 1, 1, 2, 2, 2), result);
-
- op.close();
+
+ synchronized (lock) {
+ op.close();
+ }
op.dispose();
}
catch (Exception e) {
@@ -407,8 +438,10 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
// the operator has a window time that is so long that it will not fire in this test
final long oneYear = 365L * 24 * 60 * 60 * 1000;
- AbstractAlignedProcessingTimeWindowOperator<Integer, Integer, Integer, ?> op =
- new AccumulatingProcessingTimeWindowOperator<>(validatingIdentityFunction, identitySelector,
+ AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
+ new AccumulatingProcessingTimeWindowOperator<>(
+ validatingIdentityFunction, identitySelector,
+ IntSerializer.INSTANCE, IntSerializer.INSTANCE,
oneYear, oneYear);
op.setup(mockTask, new StreamConfig(new Configuration()), out);
@@ -420,8 +453,10 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
op.processElement(new StreamRecord<Integer>(i));
}
}
-
- op.close();
+
+ synchronized (lock) {
+ op.close();
+ }
op.dispose();
// get and verify the result
@@ -450,9 +485,11 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
// the operator has a window time that is so long that it will not fire in this test
final long hundredYears = 100L * 365 * 24 * 60 * 60 * 1000;
- AbstractAlignedProcessingTimeWindowOperator<Integer, Integer, Integer, ?> op =
+ AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
new AccumulatingProcessingTimeWindowOperator<>(
- failingFunction, identitySelector, hundredYears, hundredYears);
+ failingFunction, identitySelector,
+ IntSerializer.INSTANCE, IntSerializer.INSTANCE,
+ hundredYears, hundredYears);
op.setup(mockTask, new StreamConfig(new Configuration()), out);
op.open();
@@ -464,7 +501,9 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
}
try {
- op.close();
+ synchronized (lock) {
+ op.close();
+ }
fail("This should fail with an exception");
}
catch (Exception e) {
@@ -484,12 +523,216 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
}
}
+ @Test
+ public void checkpointRestoreWithPendingWindowTumbling() {
+ final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
+ try {
+ final int windowSize = 200;
+ final CollectingOutput<Integer> out = new CollectingOutput<>(windowSize);
+ final Object lock = new Object();
+ final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
+
+ // tumbling window that triggers every 50 milliseconds
+ AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
+ new AccumulatingProcessingTimeWindowOperator<>(
+ validatingIdentityFunction, identitySelector,
+ IntSerializer.INSTANCE, IntSerializer.INSTANCE,
+ windowSize, windowSize);
+
+ op.setup(mockTask, new StreamConfig(new Configuration()), out);
+ op.open();
+
+ // inject some elements
+ final int numElementsFirst = 700;
+ for (int i = 0; i < numElementsFirst; i++) {
+ synchronized (lock) {
+ op.processElement(new StreamRecord<Integer>(i));
+ }
+ Thread.sleep(1);
+ }
+
+ // draw a snapshot and dispose the window
+ StreamTaskState state;
+ List<Integer> resultAtSnapshot;
+ synchronized (lock) {
+ int beforeSnapShot = out.getElements().size();
+ state = op.snapshotOperatorState(1L, System.currentTimeMillis());
+ resultAtSnapshot = new ArrayList<>(out.getElements());
+ int afterSnapShot = out.getElements().size();
+ assertEquals("operator performed computation during snapshot", beforeSnapShot, afterSnapShot);
+ }
+
+ // inject some random elements, which should not show up in the state
+ for (int i = 0; i < 300; i++) {
+ synchronized (lock) {
+ op.processElement(new StreamRecord<Integer>(i + numElementsFirst));
+ }
+ Thread.sleep(1);
+ }
+
+ op.dispose();
+
+ // re-create the operator and restore the state
+ final CollectingOutput<Integer> out2 = new CollectingOutput<>(windowSize);
+ op = new AccumulatingProcessingTimeWindowOperator<>(
+ validatingIdentityFunction, identitySelector,
+ IntSerializer.INSTANCE, IntSerializer.INSTANCE,
+ windowSize, windowSize);
+
+ op.setup(mockTask, new StreamConfig(new Configuration()), out2);
+ op.restoreState(state);
+ op.open();
+
+ // inject some more elements
+ final int numElements = 1000;
+ for (int i = numElementsFirst; i < numElements; i++) {
+ synchronized (lock) {
+ op.processElement(new StreamRecord<Integer>(i));
+ }
+ Thread.sleep(1);
+ }
+
+ synchronized (lock) {
+ op.close();
+ }
+ op.dispose();
+
+ // get and verify the result
+ List<Integer> finalResult = new ArrayList<>(resultAtSnapshot);
+ finalResult.addAll(out2.getElements());
+ assertEquals(numElements, finalResult.size());
+
+ Collections.sort(finalResult);
+ for (int i = 0; i < numElements; i++) {
+ assertEquals(i, finalResult.get(i).intValue());
+ }
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ finally {
+ timerService.shutdown();
+ }
+ }
+
+ @Test
+ public void checkpointRestoreWithPendingWindowSliding() {
+ final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
+ try {
+ final int factor = 4;
+ final int windowSlide = 50;
+ final int windowSize = factor * windowSlide;
+
+ final CollectingOutput<Integer> out = new CollectingOutput<>(windowSlide);
+ final Object lock = new Object();
+ final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
+
+ // sliding window (200 msecs) every 50 msecs
+ AccumulatingProcessingTimeWindowOperator<Integer, Integer, Integer> op =
+ new AccumulatingProcessingTimeWindowOperator<>(
+ validatingIdentityFunction, identitySelector,
+ IntSerializer.INSTANCE, IntSerializer.INSTANCE,
+ windowSize, windowSlide);
+
+ op.setup(mockTask, new StreamConfig(new Configuration()), out);
+ op.open();
+
+ // inject some elements
+ final int numElements = 1000;
+ final int numElementsFirst = 700;
+
+ for (int i = 0; i < numElementsFirst; i++) {
+ synchronized (lock) {
+ op.processElement(new StreamRecord<Integer>(i));
+ }
+ Thread.sleep(1);
+ }
+
+ // draw a snapshot
+ StreamTaskState state;
+ List<Integer> resultAtSnapshot;
+ synchronized (lock) {
+ int beforeSnapShot = out.getElements().size();
+ state = op.snapshotOperatorState(1L, System.currentTimeMillis());
+ resultAtSnapshot = new ArrayList<>(out.getElements());
+ int afterSnapShot = out.getElements().size();
+ assertEquals("operator performed computation during snapshot", beforeSnapShot, afterSnapShot);
+ }
+
+ assertTrue(resultAtSnapshot.size() <= factor * numElementsFirst);
+
+ // inject the remaining elements - these should not influence the snapshot
+ for (int i = numElementsFirst; i < numElements; i++) {
+ synchronized (lock) {
+ op.processElement(new StreamRecord<Integer>(i));
+ }
+ Thread.sleep(1);
+ }
+
+ op.dispose();
+
+ // re-create the operator and restore the state
+ final CollectingOutput<Integer> out2 = new CollectingOutput<>(windowSlide);
+ op = new AccumulatingProcessingTimeWindowOperator<>(
+ validatingIdentityFunction, identitySelector,
+ IntSerializer.INSTANCE, IntSerializer.INSTANCE,
+ windowSize, windowSlide);
+
+ op.setup(mockTask, new StreamConfig(new Configuration()), out2);
+ op.restoreState(state);
+ op.open();
+
+
+ // inject again the remaining elements
+ for (int i = numElementsFirst; i < numElements; i++) {
+ synchronized (lock) {
+ op.processElement(new StreamRecord<Integer>(i));
+ }
+ Thread.sleep(1);
+ }
+
+ // for a deterministic result, we need to wait until all pending triggers
+ // have fired and emitted their results
+ long deadline = System.currentTimeMillis() + 120000;
+ do {
+ Thread.sleep(20);
+ }
+ while (resultAtSnapshot.size() + out2.getElements().size() < factor * numElements
+ && System.currentTimeMillis() < deadline);
+
+ synchronized (lock) {
+ op.close();
+ }
+ op.dispose();
+
+ // get and verify the result
+ List<Integer> finalResult = new ArrayList<>(resultAtSnapshot);
+ finalResult.addAll(out2.getElements());
+ assertEquals(factor * numElements, finalResult.size());
+
+ Collections.sort(finalResult);
+ for (int i = 0; i < factor * numElements; i++) {
+ assertEquals(i / factor, finalResult.get(i).intValue());
+ }
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ finally {
+ timerService.shutdown();
+ }
+ }
+
// ------------------------------------------------------------------------
private void assertInvalidParameter(long windowSize, long windowSlide) {
try {
new AccumulatingProcessingTimeWindowOperator<String, String, String>(
- mockFunction, mockKeySelector, windowSize, windowSlide);
+ mockFunction, mockKeySelector,
+ StringSerializer.INSTANCE, StringSerializer.INSTANCE,
+ windowSize, windowSlide);
fail("This should fail with an IllegalArgumentException");
}
catch (IllegalArgumentException e) {
@@ -541,6 +784,12 @@ public class AccumulatingAlignedProcessingTimeWindowOperatorTest {
when(task.getEnvironment()).thenReturn(env);
+ // ugly java generic hacks to get the state backend into the mock
+ @SuppressWarnings("unchecked")
+ OngoingStubbing<StateBackend<?>> stubbing =
+ (OngoingStubbing<StateBackend<?>>) (OngoingStubbing<?>) when(task.getStateBackend());
+ stubbing.thenReturn(MemoryStateBackend.defaultInstance());
+
return task;
}
http://git-wip-us.apache.org/repos/asf/flink/blob/c24dca50/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java
index 106e833..ab8e551 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingAlignedProcessingTimeWindowOperatorTest.java
@@ -21,20 +21,27 @@ package org.apache.flink.streaming.runtime.operators.windowing;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.accumulators.Accumulator;
import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.common.typeutils.base.StringSerializer;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.execution.Environment;
import org.apache.flink.streaming.api.graph.StreamConfig;
import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.state.StateBackend;
+import org.apache.flink.streaming.api.state.memory.MemoryStateBackend;
import org.apache.flink.streaming.runtime.operators.Triggerable;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.apache.flink.streaming.runtime.tasks.StreamTaskState;
import org.junit.After;
import org.junit.Test;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
+import org.mockito.stubbing.OngoingStubbing;
+import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
@@ -118,25 +125,29 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
try {
AggregatingProcessingTimeWindowOperator<String, String> op;
- op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector, 5000, 1000);
+ op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
+ StringSerializer.INSTANCE, StringSerializer.INSTANCE, 5000, 1000);
assertEquals(5000, op.getWindowSize());
assertEquals(1000, op.getWindowSlide());
assertEquals(1000, op.getPaneSize());
assertEquals(5, op.getNumPanesPerWindow());
- op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector, 1000, 1000);
+ op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
+ StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1000, 1000);
assertEquals(1000, op.getWindowSize());
assertEquals(1000, op.getWindowSlide());
assertEquals(1000, op.getPaneSize());
assertEquals(1, op.getNumPanesPerWindow());
- op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector, 1500, 1000);
+ op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
+ StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1500, 1000);
assertEquals(1500, op.getWindowSize());
assertEquals(1000, op.getWindowSlide());
assertEquals(500, op.getPaneSize());
assertEquals(3, op.getNumPanesPerWindow());
- op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector, 1200, 1100);
+ op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
+ StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1200, 1100);
assertEquals(1200, op.getWindowSize());
assertEquals(1100, op.getWindowSlide());
assertEquals(100, op.getPaneSize());
@@ -157,28 +168,32 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
AggregatingProcessingTimeWindowOperator<String, String> op;
- op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector, 5000, 1000);
+ op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
+ StringSerializer.INSTANCE, StringSerializer.INSTANCE, 5000, 1000);
op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
op.open();
assertTrue(op.getNextSlideTime() % 1000 == 0);
assertTrue(op.getNextEvaluationTime() % 1000 == 0);
op.dispose();
- op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector, 1000, 1000);
+ op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
+ StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1000, 1000);
op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
op.open();
assertTrue(op.getNextSlideTime() % 1000 == 0);
assertTrue(op.getNextEvaluationTime() % 1000 == 0);
op.dispose();
- op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector, 1500, 1000);
+ op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
+ StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1500, 1000);
op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
op.open();
assertTrue(op.getNextSlideTime() % 500 == 0);
assertTrue(op.getNextEvaluationTime() % 1000 == 0);
op.dispose();
- op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector, 1200, 1100);
+ op = new AggregatingProcessingTimeWindowOperator<>(mockFunction, mockKeySelector,
+ StringSerializer.INSTANCE, StringSerializer.INSTANCE, 1200, 1100);
op.setup(mockTask, new StreamConfig(new Configuration()), mockOut);
op.open();
assertTrue(op.getNextSlideTime() % 100 == 0);
@@ -200,7 +215,9 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
AggregatingProcessingTimeWindowOperator<Integer, Integer> op =
new AggregatingProcessingTimeWindowOperator<>(
- sumFunction, identitySelector, windowSize, windowSize);
+ sumFunction, identitySelector,
+ IntSerializer.INSTANCE, IntSerializer.INSTANCE,
+ windowSize, windowSize);
final Object lock = new Object();
final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
@@ -211,11 +228,15 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
final int numElements = 1000;
for (int i = 0; i < numElements; i++) {
- op.processElement(new StreamRecord<Integer>(i));
+ synchronized (lock) {
+ op.processElement(new StreamRecord<Integer>(i));
+ }
Thread.sleep(1);
}
- op.close();
+ synchronized (lock) {
+ op.close();
+ }
op.dispose();
// get and verify the result
@@ -238,7 +259,6 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
@Test
public void testTumblingWindowDuplicateElements() {
-
final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
try {
@@ -250,7 +270,9 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
AggregatingProcessingTimeWindowOperator<Integer, Integer> op =
new AggregatingProcessingTimeWindowOperator<>(
- sumFunction, identitySelector, windowSize, windowSize);
+ sumFunction, identitySelector,
+ IntSerializer.INSTANCE, IntSerializer.INSTANCE,
+ windowSize, windowSize);
op.setup(mockTask, new StreamConfig(new Configuration()), out);
op.open();
@@ -261,22 +283,23 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
int window = 1;
while (window <= numWindows) {
- long nextTime = op.getNextEvaluationTime();
- int val = ((int) nextTime) ^ ((int) (nextTime >>> 32));
-
synchronized (lock) {
+ long nextTime = op.getNextEvaluationTime();
+ int val = ((int) nextTime) ^ ((int) (nextTime >>> 32));
+
op.processElement(new StreamRecord<Integer>(val));
+
+ if (nextTime != previousNextTime) {
+ window++;
+ previousNextTime = nextTime;
+ }
}
-
- if (nextTime != previousNextTime) {
- window++;
- previousNextTime = nextTime;
- }
-
Thread.sleep(1);
}
- op.close();
+ synchronized (lock) {
+ op.close();
+ }
op.dispose();
List<Integer> result = out.getElements();
@@ -287,12 +310,13 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
// deduplicate for more accurate checks
HashSet<Integer> set = new HashSet<>(result);
- assertTrue(set.size() == 10 || set.size() == 11);
+ assertTrue(set.size() == 10);
}
catch (Exception e) {
e.printStackTrace();
fail(e.getMessage());
- } finally {
+ }
+ finally {
timerService.shutdown();
}
}
@@ -308,7 +332,10 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
// tumbling window that triggers every 20 milliseconds
AggregatingProcessingTimeWindowOperator<Integer, Integer> op =
- new AggregatingProcessingTimeWindowOperator<>(sumFunction, identitySelector, 150, 50);
+ new AggregatingProcessingTimeWindowOperator<>(
+ sumFunction, identitySelector,
+ IntSerializer.INSTANCE, IntSerializer.INSTANCE,
+ 150, 50);
op.setup(mockTask, new StreamConfig(new Configuration()), out);
op.open();
@@ -322,7 +349,9 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
Thread.sleep(1);
}
- op.close();
+ synchronized (lock) {
+ op.close();
+ }
op.dispose();
// get and verify the result
@@ -369,7 +398,9 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
// tumbling window that triggers every 20 milliseconds
AggregatingProcessingTimeWindowOperator<Integer, Integer> op =
- new AggregatingProcessingTimeWindowOperator<>(sumFunction, identitySelector, 150, 50);
+ new AggregatingProcessingTimeWindowOperator<>(
+ sumFunction, identitySelector,
+ IntSerializer.INSTANCE, IntSerializer.INSTANCE, 150, 50);
op.setup(mockTask, new StreamConfig(new Configuration()), out);
op.open();
@@ -388,8 +419,10 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
Collections.sort(result);
assertEquals(Arrays.asList(1, 1, 1, 2, 2, 2), result);
-
- op.close();
+
+ synchronized (lock) {
+ op.close();
+ }
op.dispose();
}
catch (Exception e) {
@@ -412,7 +445,8 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
// the operator has a window time that is so long that it will not fire in this test
final long oneYear = 365L * 24 * 60 * 60 * 1000;
AggregatingProcessingTimeWindowOperator<Integer, Integer> op =
- new AggregatingProcessingTimeWindowOperator<>(sumFunction, identitySelector, oneYear, oneYear);
+ new AggregatingProcessingTimeWindowOperator<>(sumFunction, identitySelector,
+ IntSerializer.INSTANCE, IntSerializer.INSTANCE, oneYear, oneYear);
op.setup(mockTask, new StreamConfig(new Configuration()), out);
op.open();
@@ -423,8 +457,10 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
op.processElement(new StreamRecord<Integer>(i));
}
}
-
- op.close();
+
+ synchronized (lock) {
+ op.close();
+ }
op.dispose();
// get and verify the result
@@ -455,7 +491,9 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
final long hundredYears = 100L * 365 * 24 * 60 * 60 * 1000;
AggregatingProcessingTimeWindowOperator<Integer, Integer> op =
new AggregatingProcessingTimeWindowOperator<>(
- failingFunction, identitySelector, hundredYears, hundredYears);
+ failingFunction, identitySelector,
+ IntSerializer.INSTANCE, IntSerializer.INSTANCE,
+ hundredYears, hundredYears);
op.setup(mockTask, new StreamConfig(new Configuration()), out);
op.open();
@@ -484,13 +522,220 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
timerService.shutdown();
}
}
+
+ @Test
+ public void checkpointRestoreWithPendingWindowTumbling() {
+ final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
+ try {
+ final int windowSize = 200;
+ final CollectingOutput<Integer> out = new CollectingOutput<>(windowSize);
+ final Object lock = new Object();
+ final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
+
+ // tumbling window that triggers every 50 milliseconds
+ AggregatingProcessingTimeWindowOperator<Integer, Integer> op =
+ new AggregatingProcessingTimeWindowOperator<>(
+ sumFunction, identitySelector,
+ IntSerializer.INSTANCE, IntSerializer.INSTANCE,
+ windowSize, windowSize);
+
+ op.setup(mockTask, new StreamConfig(new Configuration()), out);
+ op.open();
+
+ // inject some elements
+ final int numElementsFirst = 700;
+ final int numElements = 1000;
+
+ for (int i = 0; i < numElementsFirst; i++) {
+ synchronized (lock) {
+ op.processElement(new StreamRecord<Integer>(i));
+ }
+ Thread.sleep(1);
+ }
+
+ // draw a snapshot and dispose the window
+ StreamTaskState state;
+ List<Integer> resultAtSnapshot;
+ synchronized (lock) {
+ int beforeSnapShot = out.getElements().size();
+ state = op.snapshotOperatorState(1L, System.currentTimeMillis());
+ resultAtSnapshot = new ArrayList<>(out.getElements());
+ int afterSnapShot = out.getElements().size();
+ assertEquals("operator performed computation during snapshot", beforeSnapShot, afterSnapShot);
+ }
+
+ assertTrue(resultAtSnapshot.size() <= numElementsFirst);
+
+ // inject some random elements, which should not show up in the state
+ for (int i = numElementsFirst; i < numElements; i++) {
+ synchronized (lock) {
+ op.processElement(new StreamRecord<Integer>(i));
+ }
+ Thread.sleep(1);
+ }
+
+ op.dispose();
+
+ // re-create the operator and restore the state
+ final CollectingOutput<Integer> out2 = new CollectingOutput<>(windowSize);
+ op = new AggregatingProcessingTimeWindowOperator<>(
+ sumFunction, identitySelector,
+ IntSerializer.INSTANCE, IntSerializer.INSTANCE,
+ windowSize, windowSize);
+
+ op.setup(mockTask, new StreamConfig(new Configuration()), out2);
+ op.restoreState(state);
+ op.open();
+
+ // inject the remaining elements
+ for (int i = numElementsFirst; i < numElements; i++) {
+ synchronized (lock) {
+ op.processElement(new StreamRecord<Integer>(i));
+ }
+ Thread.sleep(1);
+ }
+
+ synchronized (lock) {
+ op.close();
+ }
+ op.dispose();
+
+ // get and verify the result
+ List<Integer> finalResult = new ArrayList<>(resultAtSnapshot);
+ finalResult.addAll(out2.getElements());
+ assertEquals(numElements, finalResult.size());
+
+ Collections.sort(finalResult);
+ for (int i = 0; i < numElements; i++) {
+ assertEquals(i, finalResult.get(i).intValue());
+ }
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ finally {
+ timerService.shutdown();
+ }
+ }
+
+ @Test
+ public void checkpointRestoreWithPendingWindowSliding() {
+ final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
+ try {
+ final int factor = 4;
+ final int windowSlide = 50;
+ final int windowSize = factor * windowSlide;
+
+ final CollectingOutput<Integer> out = new CollectingOutput<>(windowSlide);
+ final Object lock = new Object();
+ final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
+
+ // sliding window (200 msecs) every 50 msecs
+ AggregatingProcessingTimeWindowOperator<Integer, Integer> op =
+ new AggregatingProcessingTimeWindowOperator<>(
+ sumFunction, identitySelector,
+ IntSerializer.INSTANCE, IntSerializer.INSTANCE,
+ windowSize, windowSlide);
+
+ op.setup(mockTask, new StreamConfig(new Configuration()), out);
+ op.open();
+
+ // inject some elements
+ final int numElements = 1000;
+ final int numElementsFirst = 700;
+
+ for (int i = 0; i < numElementsFirst; i++) {
+ synchronized (lock) {
+ op.processElement(new StreamRecord<Integer>(i));
+ }
+ Thread.sleep(1);
+ }
+
+ // draw a snapshot
+ StreamTaskState state;
+ List<Integer> resultAtSnapshot;
+ synchronized (lock) {
+ int beforeSnapShot = out.getElements().size();
+ state = op.snapshotOperatorState(1L, System.currentTimeMillis());
+ resultAtSnapshot = new ArrayList<>(out.getElements());
+ int afterSnapShot = out.getElements().size();
+ assertEquals("operator performed computation during snapshot", beforeSnapShot, afterSnapShot);
+ }
+
+ assertTrue(resultAtSnapshot.size() <= factor * numElementsFirst);
+
+ // inject the remaining elements - these should not influence the snapshot
+ for (int i = numElementsFirst; i < numElements; i++) {
+ synchronized (lock) {
+ op.processElement(new StreamRecord<Integer>(i));
+ }
+ Thread.sleep(1);
+ }
+
+ op.dispose();
+
+ // re-create the operator and restore the state
+ final CollectingOutput<Integer> out2 = new CollectingOutput<>(windowSlide);
+ op = new AggregatingProcessingTimeWindowOperator<>(
+ sumFunction, identitySelector,
+ IntSerializer.INSTANCE, IntSerializer.INSTANCE,
+ windowSize, windowSlide);
+
+ op.setup(mockTask, new StreamConfig(new Configuration()), out2);
+ op.restoreState(state);
+ op.open();
+
+
+ // inject again the remaining elements
+ for (int i = numElementsFirst; i < numElements; i++) {
+ synchronized (lock) {
+ op.processElement(new StreamRecord<Integer>(i));
+ }
+ Thread.sleep(1);
+ }
+
+ // for a deterministic result, we need to wait until all pending triggers
+ // have fired and emitted their results
+ long deadline = System.currentTimeMillis() + 120000;
+ do {
+ Thread.sleep(20);
+ }
+ while (resultAtSnapshot.size() + out2.getElements().size() < factor * numElements
+ && System.currentTimeMillis() < deadline);
+
+ synchronized (lock) {
+ op.close();
+ }
+ op.dispose();
+
+ // get and verify the result
+ List<Integer> finalResult = new ArrayList<>(resultAtSnapshot);
+ finalResult.addAll(out2.getElements());
+ assertEquals(factor * numElements, finalResult.size());
+
+ Collections.sort(finalResult);
+ for (int i = 0; i < factor * numElements; i++) {
+ assertEquals(i / factor, finalResult.get(i).intValue());
+ }
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ finally {
+ timerService.shutdown();
+ }
+ }
// ------------------------------------------------------------------------
private void assertInvalidParameter(long windowSize, long windowSlide) {
try {
new AggregatingProcessingTimeWindowOperator<String, String>(
- mockFunction, mockKeySelector, windowSize, windowSlide);
+ mockFunction, mockKeySelector,
+ StringSerializer.INSTANCE, StringSerializer.INSTANCE,
+ windowSize, windowSlide);
fail("This should fail with an IllegalArgumentException");
}
catch (IllegalArgumentException e) {
@@ -537,6 +782,12 @@ public class AggregatingAlignedProcessingTimeWindowOperatorTest {
when(env.getUserClassLoader()).thenReturn(AggregatingAlignedProcessingTimeWindowOperatorTest.class.getClassLoader());
when(task.getEnvironment()).thenReturn(env);
+
+ // ugly java generic hacks to get the state backend into the mock
+ @SuppressWarnings("unchecked")
+ OngoingStubbing<StateBackend<?>> stubbing =
+ (OngoingStubbing<StateBackend<?>>) (OngoingStubbing<?>) when(task.getStateBackend());
+ stubbing.thenReturn(MemoryStateBackend.defaultInstance());
return task;
}
http://git-wip-us.apache.org/repos/asf/flink/blob/c24dca50/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java
index 000a1a2..81d3a69 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/MockContext.java
@@ -28,17 +28,22 @@ import java.util.concurrent.TimeUnit;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.accumulators.Accumulator;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.operators.testutils.MockEnvironment;
import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider;
import org.apache.flink.streaming.api.graph.StreamConfig;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.state.StateBackend;
+import org.apache.flink.streaming.api.state.memory.MemoryStateBackend;
import org.apache.flink.streaming.runtime.operators.Triggerable;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.tasks.StreamTask;
import org.mockito.invocation.InvocationOnMock;
import org.mockito.stubbing.Answer;
+import org.mockito.stubbing.OngoingStubbing;
import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyLong;
@@ -48,13 +53,11 @@ import static org.mockito.Mockito.when;
public class MockContext<IN, OUT> {
- private Collection<IN> inputs;
private List<OUT> outputs;
private MockOutput<OUT> output;
public MockContext(Collection<IN> inputs) {
- this.inputs = inputs;
if (inputs.isEmpty()) {
throw new RuntimeException("Inputs must not be empty");
}
@@ -72,20 +75,35 @@ public class MockContext<IN, OUT> {
}
public static <IN, OUT> List<OUT> createAndExecute(OneInputStreamOperator<IN, OUT> operator, List<IN> inputs) {
+ return createAndExecuteForKeyedStream(operator, inputs, null, null);
+ }
+
+ public static <IN, OUT, KEY> List<OUT> createAndExecuteForKeyedStream(
+ OneInputStreamOperator<IN, OUT> operator, List<IN> inputs,
+ KeySelector<IN, KEY> keySelector, TypeInformation<KEY> keyType) {
+
MockContext<IN, OUT> mockContext = new MockContext<IN, OUT>(inputs);
+ StreamConfig config = new StreamConfig(new Configuration());
+ if (keySelector != null && keyType != null) {
+ config.setStateKeySerializer(keyType.createSerializer(new ExecutionConfig()));
+ config.setStatePartitioner(keySelector);
+ }
+
final ScheduledExecutorService timerService = Executors.newSingleThreadScheduledExecutor();
final Object lock = new Object();
final StreamTask<?, ?> mockTask = createMockTaskWithTimer(timerService, lock);
- operator.setup(mockTask, new StreamConfig(new Configuration()), mockContext.output);
+ operator.setup(mockTask, config, mockContext.output);
try {
operator.open();
- StreamRecord<IN> nextRecord;
+ StreamRecord<IN> record = new StreamRecord<IN>(null);
for (IN in: inputs) {
+ record = record.replace(in);
synchronized (lock) {
- operator.processElement(new StreamRecord<IN>(in));
+ operator.setKeyContextElement(record);
+ operator.processElement(record);
}
}
@@ -130,6 +148,12 @@ public class MockContext<IN, OUT> {
}
}).when(task).registerTimer(anyLong(), any(Triggerable.class));
+ // ugly Java generic hacks to get the generic state backend into the mock
+ @SuppressWarnings("unchecked")
+ OngoingStubbing<StateBackend<?>> stubbing =
+ (OngoingStubbing<StateBackend<?>>) (OngoingStubbing<?>) when(task.getStateBackend());
+ stubbing.thenReturn(MemoryStateBackend.defaultInstance());
+
return task;
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/c24dca50/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
index edf3a09..b83feca 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/OneInputStreamOperatorTestHarness.java
@@ -18,7 +18,10 @@
package org.apache.flink.streaming.util;
import org.apache.flink.api.common.ExecutionConfig;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.java.ClosureCleaner;
+import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.api.java.typeutils.TypeExtractor;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.execution.Environment;
@@ -54,17 +57,17 @@ public class OneInputStreamOperatorTestHarness<IN, OUT> {
final ConcurrentLinkedQueue<Object> outputList;
+ final StreamConfig config;
+
final ExecutionConfig executionConfig;
final Object checkpointLock;
-
- public OneInputStreamOperatorTestHarness(OneInputStreamOperator<IN, OUT> operator) {
- this(operator, new StreamConfig(new Configuration()));
- }
- public OneInputStreamOperatorTestHarness(OneInputStreamOperator<IN, OUT> operator, StreamConfig config) {
+
+ public OneInputStreamOperatorTestHarness(OneInputStreamOperator<IN, OUT> operator) {
this.operator = operator;
this.outputList = new ConcurrentLinkedQueue<Object>();
+ this.config = new StreamConfig(new Configuration());
this.executionConfig = new ExecutionConfig();
this.checkpointLock = new Object();
@@ -82,9 +85,15 @@ public class OneInputStreamOperatorTestHarness<IN, OUT> {
(OngoingStubbing<StateBackend<?>>) (OngoingStubbing<?>) when(mockTask.getStateBackend());
stubbing.thenReturn(MemoryStateBackend.defaultInstance());
- operator.setup(mockTask, new StreamConfig(new Configuration()), new MockOutput());
+ operator.setup(mockTask, config, new MockOutput());
}
+ public <K> void configureForKeyedStream(KeySelector<IN, K> keySelector, TypeInformation<K> keyType) {
+ ClosureCleaner.clean(keySelector, false);
+ config.setStatePartitioner(keySelector);
+ config.setStateKeySerializer(keyType.createSerializer(executionConfig));
+ }
+
/**
* Get all the output from the task. This contains StreamRecords and Events interleaved. Use
* {@link org.apache.flink.streaming.util.TestHarnessUtil#getStreamRecordsFromOutput(java.util.List)}
@@ -109,11 +118,13 @@ public class OneInputStreamOperatorTestHarness<IN, OUT> {
}
public void processElement(StreamRecord<IN> element) throws Exception {
+ operator.setKeyContextElement(element);
operator.processElement(element);
}
public void processElements(Collection<StreamRecord<IN>> elements) throws Exception {
for (StreamRecord<IN> element: elements) {
+ operator.setKeyContextElement(element);
operator.processElement(element);
}
}
@@ -127,13 +138,11 @@ public class OneInputStreamOperatorTestHarness<IN, OUT> {
private TypeSerializer<OUT> outputSerializer;
@Override
- @SuppressWarnings("unchecked")
public void emitWatermark(Watermark mark) {
outputList.add(mark);
}
@Override
- @SuppressWarnings("unchecked")
public void collect(StreamRecord<OUT> element) {
if (outputSerializer == null) {
outputSerializer = TypeExtractor.getForObject(element.getValue()).createSerializer(executionConfig);
[03/24] flink git commit: [FLINK-2808] [streaming] Refactor and
extend state backend abstraction
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CoStreamCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CoStreamCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CoStreamCheckpointingITCase.java
index e5a1c23..c503a1f 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CoStreamCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CoStreamCheckpointingITCase.java
@@ -21,7 +21,6 @@ package org.apache.flink.test.checkpointing;
import org.apache.flink.api.common.functions.RichFilterFunction;
import org.apache.flink.api.common.functions.RichMapFunction;
import org.apache.flink.api.common.functions.RichReduceFunction;
-import org.apache.flink.api.common.state.OperatorState;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.checkpoint.Checkpointed;
import org.apache.flink.streaming.api.datastream.DataStream;
@@ -48,11 +47,6 @@ import static org.junit.Assert.assertTrue;
* this barriers are correctly forwarded.
*
* <p>
- * This uses a mixture of Operators with the {@link Checkpointed} interface and the new
- * {@link org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext#getOperatorState}
- * method.
- *
- * <p>
* The test triggers a failure after a while and verifies that, after completion, the
* state reflects the "exactly once" semantics.
*/
@@ -142,25 +136,21 @@ public class CoStreamCheckpointingITCase extends StreamFaultToleranceTestBase {
// --------------------------------------------------------------------------------------------
private static class StringGeneratingSourceFunction extends RichSourceFunction<String>
- implements ParallelSourceFunction<String> {
+ implements ParallelSourceFunction<String>, Checkpointed<Integer> {
+ static final long[] counts = new long[PARALLELISM];
+
private final long numElements;
private Random rnd;
private StringBuilder stringBuilder;
- private OperatorState<Integer> index;
+ private int index;
private int step;
- private volatile boolean isRunning;
-
- static final long[] counts = new long[PARALLELISM];
- @Override
- public void close() throws IOException {
- counts[getRuntimeContext().getIndexOfThisSubtask()] = index.value();
- }
-
+ private volatile boolean isRunning = true;
+
StringGeneratingSourceFunction(long numElements) {
this.numElements = numElements;
}
@@ -169,20 +159,19 @@ public class CoStreamCheckpointingITCase extends StreamFaultToleranceTestBase {
public void open(Configuration parameters) throws IOException {
rnd = new Random();
stringBuilder = new StringBuilder();
+
step = getRuntimeContext().getNumberOfParallelSubtasks();
-
-
- index = getRuntimeContext().getOperatorState("index", getRuntimeContext().getIndexOfThisSubtask(), false);
-
- isRunning = true;
+ if (index == 0) {
+ index = getRuntimeContext().getIndexOfThisSubtask();
+ }
}
@Override
public void run(SourceContext<String> ctx) throws Exception {
final Object lockingObject = ctx.getCheckpointLock();
- while (isRunning && index.value() < numElements) {
- char first = (char) ((index.value() % 40) + 40);
+ while (isRunning && index < numElements) {
+ char first = (char) ((index % 40) + 40);
stringBuilder.setLength(0);
stringBuilder.append(first);
@@ -190,13 +179,18 @@ public class CoStreamCheckpointingITCase extends StreamFaultToleranceTestBase {
String result = randomString(stringBuilder, rnd);
synchronized (lockingObject) {
- index.update(index.value() + step);
+ index += step;
ctx.collect(result);
}
}
}
@Override
+ public void close() throws IOException {
+ counts[getRuntimeContext().getIndexOfThisSubtask()] = index;
+ }
+
+ @Override
public void cancel() {
isRunning = false;
}
@@ -211,29 +205,46 @@ public class CoStreamCheckpointingITCase extends StreamFaultToleranceTestBase {
return bld.toString();
}
+
+ @Override
+ public Integer snapshotState(long checkpointId, long checkpointTimestamp) {
+ return index;
+ }
+
+ @Override
+ public void restoreState(Integer state) {
+ index = state;
+ }
}
- private static class StatefulCounterFunction extends RichMapFunction<PrefixCount, PrefixCount> {
+ private static class StatefulCounterFunction extends RichMapFunction<PrefixCount, PrefixCount>
+ implements Checkpointed<Long> {
- private OperatorState<Long> count;
static final long[] counts = new long[PARALLELISM];
+
+ private long count;
+
@Override
public PrefixCount map(PrefixCount value) throws Exception {
- count.update(count.value() + 1);
+ count++;
return value;
}
@Override
- public void open(Configuration conf) throws IOException {
- count = getRuntimeContext().getOperatorState("count", 0L, false);
+ public void close() throws IOException {
+ counts[getRuntimeContext().getIndexOfThisSubtask()] = count;
}
@Override
- public void close() throws IOException {
- counts[getRuntimeContext().getIndexOfThisSubtask()] = count.value();
+ public Long snapshotState(long checkpointId, long checkpointTimestamp) {
+ return count;
}
+ @Override
+ public void restoreState(Long state) {
+ count = state;
+ }
}
private static class OnceFailingReducer extends RichReduceFunction<PrefixCount> {
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/PartitionedStateCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/PartitionedStateCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/PartitionedStateCheckpointingITCase.java
index 108e1e6..0fcedda 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/PartitionedStateCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/PartitionedStateCheckpointingITCase.java
@@ -29,10 +29,10 @@ import java.util.concurrent.ConcurrentHashMap;
import org.apache.flink.api.common.functions.RichMapFunction;
import org.apache.flink.api.common.state.OperatorState;
-import org.apache.flink.api.common.state.StateCheckpointer;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.checkpoint.Checkpointed;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
@@ -83,20 +83,21 @@ public class PartitionedStateCheckpointingITCase extends StreamFaultToleranceTes
// Custom Functions
// --------------------------------------------------------------------------------------------
- private static class IntGeneratingSourceFunction extends RichParallelSourceFunction<Integer> {
+ private static class IntGeneratingSourceFunction extends RichParallelSourceFunction<Integer>
+ implements Checkpointed<Integer> {
private final long numElements;
- private OperatorState<Integer> index;
+ private int index;
private int step;
- private volatile boolean isRunning;
+ private volatile boolean isRunning = true;
static final long[] counts = new long[PARALLELISM];
@Override
public void close() throws IOException {
- counts[getRuntimeContext().getIndexOfThisSubtask()] = index.value();
+ counts[getRuntimeContext().getIndexOfThisSubtask()] = index;
}
IntGeneratingSourceFunction(long numElements) {
@@ -106,22 +107,18 @@ public class PartitionedStateCheckpointingITCase extends StreamFaultToleranceTes
@Override
public void open(Configuration parameters) throws IOException {
step = getRuntimeContext().getNumberOfParallelSubtasks();
-
- index = getRuntimeContext().getOperatorState("index",
- getRuntimeContext().getIndexOfThisSubtask(), false);
-
- isRunning = true;
+ index = getRuntimeContext().getIndexOfThisSubtask();
}
@Override
public void run(SourceContext<Integer> ctx) throws Exception {
final Object lockingObject = ctx.getCheckpointLock();
- while (isRunning && index.value() < numElements) {
+ while (isRunning && index < numElements) {
synchronized (lockingObject) {
- index.update(index.value() + step);
- ctx.collect(index.value() % 40);
+ index += step;
+ ctx.collect(index % 40);
}
}
}
@@ -130,11 +127,22 @@ public class PartitionedStateCheckpointingITCase extends StreamFaultToleranceTes
public void cancel() {
isRunning = false;
}
+
+ @Override
+ public Integer snapshotState(long checkpointId, long checkpointTimestamp) {
+ return index;
+ }
+
+ @Override
+ public void restoreState(Integer state) {
+ index = state;
+ }
}
private static class OnceFailingPartitionedSum extends RichMapFunction<Integer, Tuple2<Integer, Long>> {
private static Map<Integer, Long> allSums = new ConcurrentHashMap<Integer, Long>();
+
private static volatile boolean hasFailed = false;
private final long numElements;
@@ -157,7 +165,7 @@ public class PartitionedStateCheckpointingITCase extends StreamFaultToleranceTes
failurePos = (new Random().nextLong() % (failurePosMax - failurePosMin)) + failurePosMin;
count = 0;
- sum = getRuntimeContext().getOperatorState("sum", 0L, true);
+ sum = getRuntimeContext().getKeyValueState(Long.class, 0L);
}
@Override
@@ -183,20 +191,7 @@ public class PartitionedStateCheckpointingITCase extends StreamFaultToleranceTes
@Override
public void open(Configuration parameters) throws IOException {
- counts = getRuntimeContext().getOperatorState("count", NonSerializableLong.of(0L), true,
- new StateCheckpointer<NonSerializableLong, String>() {
-
- @Override
- public String snapshotState(NonSerializableLong state, long id, long ts) {
- return state.value.toString();
- }
-
- @Override
- public NonSerializableLong restoreState(String stateSnapshot) {
- return NonSerializableLong.of(Long.parseLong(stateSnapshot));
- }
-
- });
+ counts = getRuntimeContext().getKeyValueState(NonSerializableLong.class, NonSerializableLong.of(0L));
}
@Override
@@ -204,7 +199,6 @@ public class PartitionedStateCheckpointingITCase extends StreamFaultToleranceTes
long currentCount = counts.value().value + 1;
counts.update(NonSerializableLong.of(currentCount));
allCounts.put(value.f0, currentCount);
-
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java
index 8a75de5..f517f83 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpoinedITCase.java
@@ -265,28 +265,33 @@ public class StateCheckpoinedITCase extends StreamFaultToleranceTestBase {
}
}
- private static class StatefulCounterFunction extends RichMapFunction<PrefixCount, PrefixCount> {
+ private static class StatefulCounterFunction extends RichMapFunction<PrefixCount, PrefixCount>
+ implements Checkpointed<Long> {
static final long[] counts = new long[PARALLELISM];
- private OperatorState<Long> count;
+ private long count;
@Override
public PrefixCount map(PrefixCount value) throws Exception {
- count.update(count.value() + 1);
+ count++;
return value;
}
@Override
- public void open(Configuration conf) throws IOException {
- count = getRuntimeContext().getOperatorState("count", 0L, false);
+ public void close() throws IOException {
+ counts[getRuntimeContext().getIndexOfThisSubtask()] = count;
}
@Override
- public void close() throws IOException {
- counts[getRuntimeContext().getIndexOfThisSubtask()] = count.value();
+ public Long snapshotState(long checkpointId, long checkpointTimestamp) {
+ return count;
+ }
+
+ @Override
+ public void restoreState(Long state) {
+ count = state;
}
-
}
private static class OnceFailingAggregator extends RichFlatMapFunction<PrefixCount, PrefixCount>
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java
index 270cfaa..08af93a 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java
@@ -21,7 +21,6 @@ package org.apache.flink.test.checkpointing;
import org.apache.flink.api.common.functions.RichFilterFunction;
import org.apache.flink.api.common.functions.RichMapFunction;
import org.apache.flink.api.common.functions.RichReduceFunction;
-import org.apache.flink.api.common.state.OperatorState;
import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier;
@@ -101,14 +100,16 @@ public class StreamCheckpointNotifierITCase extends StreamFaultToleranceTestBase
@Override
public void postSubmit() {
- List[][] checkList = new List[][]{ GeneratingSourceFunction.completedCheckpoints,
+ @SuppressWarnings({"unchecked", "rawtypes"})
+ List<Long>[][] checkList = new List[][] {
+ GeneratingSourceFunction.completedCheckpoints,
IdentityMapFunction.completedCheckpoints,
LongRichFilterFunction.completedCheckpoints,
LeftIdentityCoRichFlatMapFunction.completedCheckpoints};
long failureCheckpointID = OnceFailingReducer.failureCheckpointID;
- for(List[] parallelNotifications : checkList) {
+ for(List<Long>[] parallelNotifications : checkList) {
for (int i = 0; i < PARALLELISM; i++){
List<Long> notifications = parallelNotifications[i];
assertTrue("No checkpoint notification was received.",
@@ -134,21 +135,23 @@ public class StreamCheckpointNotifierITCase extends StreamFaultToleranceTestBase
* interface it stores all the checkpoint ids it has seen in a static list.
*/
private static class GeneratingSourceFunction extends RichSourceFunction<Long>
- implements ParallelSourceFunction<Long>, CheckpointNotifier {
+ implements ParallelSourceFunction<Long>, CheckpointNotifier, Checkpointed<Integer> {
- // operator life cycle
- private volatile boolean isRunning;
+ @SuppressWarnings({"unchecked", "rawtypes"})
+ static List<Long>[] completedCheckpoints = new List[PARALLELISM];
+
// operator behaviour
private final long numElements;
private long result;
- private OperatorState<Integer> index;
+ private int index;
private int step;
// test behaviour
private int subtaskId;
- public static List[] completedCheckpoints = new List[PARALLELISM];
+
+ private volatile boolean isRunning = true;
GeneratingSourceFunction(long numElements) {
this.numElements = numElements;
@@ -158,26 +161,27 @@ public class StreamCheckpointNotifierITCase extends StreamFaultToleranceTestBase
public void open(Configuration parameters) throws IOException {
step = getRuntimeContext().getNumberOfParallelSubtasks();
subtaskId = getRuntimeContext().getIndexOfThisSubtask();
- index = getRuntimeContext().getOperatorState("index", subtaskId, false);
+
+ if (index == 0) {
+ index = subtaskId;
+ }
// Create a collection on the first open
if (completedCheckpoints[subtaskId] == null) {
- completedCheckpoints[subtaskId] = new ArrayList();
+ completedCheckpoints[subtaskId] = new ArrayList<>();
}
-
- isRunning = true;
}
@Override
public void run(SourceContext<Long> ctx) throws Exception {
final Object lockingObject = ctx.getCheckpointLock();
- while (isRunning && index.value() < numElements) {
+ while (isRunning && index < numElements) {
- result = index.value() % 10;
+ result = index % 10;
synchronized (lockingObject) {
- index.update(index.value() + step);
+ index += step;
ctx.collect(result);
}
}
@@ -192,6 +196,16 @@ public class StreamCheckpointNotifierITCase extends StreamFaultToleranceTestBase
public void notifyCheckpointComplete(long checkpointId) throws Exception {
completedCheckpoints[subtaskId].add(checkpointId);
}
+
+ @Override
+ public Integer snapshotState(long checkpointId, long checkpointTimestamp) {
+ return index;
+ }
+
+ @Override
+ public void restoreState(Integer state) {
+ index = state;
+ }
}
/**
@@ -201,7 +215,9 @@ public class StreamCheckpointNotifierITCase extends StreamFaultToleranceTestBase
private static class IdentityMapFunction extends RichMapFunction<Long, Tuple1<Long>>
implements CheckpointNotifier {
- public static List[] completedCheckpoints = new List[PARALLELISM];
+ @SuppressWarnings({"unchecked", "rawtypes"})
+ public static List<Long>[] completedCheckpoints = new List[PARALLELISM];
+
private int subtaskId;
@Override
@@ -215,7 +231,7 @@ public class StreamCheckpointNotifierITCase extends StreamFaultToleranceTestBase
// Create a collection on the first open
if (completedCheckpoints[subtaskId] == null) {
- completedCheckpoints[subtaskId] = new ArrayList();
+ completedCheckpoints[subtaskId] = new ArrayList<>();
}
}
@@ -283,7 +299,9 @@ public class StreamCheckpointNotifierITCase extends StreamFaultToleranceTestBase
private static class LongRichFilterFunction extends RichFilterFunction<Long>
implements CheckpointNotifier {
- public static List[] completedCheckpoints = new List[PARALLELISM];
+ @SuppressWarnings({"unchecked", "rawtypes"})
+ static List<Long>[] completedCheckpoints = new List[PARALLELISM];
+
private int subtaskId;
@Override
@@ -297,7 +315,7 @@ public class StreamCheckpointNotifierITCase extends StreamFaultToleranceTestBase
// Create a collection on the first open
if (completedCheckpoints[subtaskId] == null) {
- completedCheckpoints[subtaskId] = new ArrayList();
+ completedCheckpoints[subtaskId] = new ArrayList<>();
}
}
@@ -315,7 +333,8 @@ public class StreamCheckpointNotifierITCase extends StreamFaultToleranceTestBase
private static class LeftIdentityCoRichFlatMapFunction extends RichCoFlatMapFunction<Long, Long, Long>
implements CheckpointNotifier {
- public static List[] completedCheckpoints = new List[PARALLELISM];
+ @SuppressWarnings({"unchecked", "rawtypes"})
+ public static List<Long>[] completedCheckpoints = new List[PARALLELISM];
private int subtaskId;
@Override
@@ -324,7 +343,7 @@ public class StreamCheckpointNotifierITCase extends StreamFaultToleranceTestBase
// Create a collection on the first open
if (completedCheckpoints[subtaskId] == null) {
- completedCheckpoints[subtaskId] = new ArrayList();
+ completedCheckpoints[subtaskId] = new ArrayList<>();
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java
index 0804d53..992a679 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java
@@ -21,7 +21,6 @@ package org.apache.flink.test.checkpointing;
import org.apache.flink.api.common.functions.RichFilterFunction;
import org.apache.flink.api.common.functions.RichMapFunction;
import org.apache.flink.api.common.state.OperatorState;
-import org.apache.flink.api.common.state.StateCheckpointer;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.checkpoint.Checkpointed;
import org.apache.flink.streaming.api.datastream.DataStream;
@@ -36,7 +35,6 @@ import java.util.Random;
import java.util.concurrent.ConcurrentHashMap;
import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
/**
* A simple test that runs a streaming topology with checkpointing enabled.
@@ -62,14 +60,15 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
stream
// -------------- first vertex, chained to the source ----------------
- .filter(new StringRichFilterFunction()).shuffle()
+ .filter(new StringRichFilterFunction())
+ .shuffle()
// -------------- seconds vertex - the stateful one that also fails ----------------
.map(new StringPrefixCountRichMapFunction())
.startNewChain()
.map(new StatefulCounterFunction())
- // -------------- third vertex - counter and the sink ----------------
+ // -------------- third vertex - counter and the sink ----------------
.keyBy("prefix")
.map(new OnceFailingPrefixCounter(NUM_STRINGS))
.addSink(new SinkFunction<PrefixCount>() {
@@ -118,22 +117,23 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
// --------------------------------------------------------------------------------------------
private static class StringGeneratingSourceFunction extends RichSourceFunction<String>
- implements ParallelSourceFunction<String> {
+ implements ParallelSourceFunction<String>, Checkpointed<Integer> {
private final long numElements;
- private Random rnd;
- private StringBuilder stringBuilder;
+ private final Random rnd = new Random();
+ private final StringBuilder stringBuilder = new StringBuilder();
- private OperatorState<Integer> index;
+ private int index;
private int step;
- private volatile boolean isRunning;
+ private volatile boolean isRunning = true;
static final long[] counts = new long[PARALLELISM];
+
@Override
public void close() throws IOException {
- counts[getRuntimeContext().getIndexOfThisSubtask()] = index.value();
+ counts[getRuntimeContext().getIndexOfThisSubtask()] = index;
}
@@ -143,22 +143,18 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
@Override
public void open(Configuration parameters) throws IOException {
- rnd = new Random();
- stringBuilder = new StringBuilder();
step = getRuntimeContext().getNumberOfParallelSubtasks();
-
-
- index = getRuntimeContext().getOperatorState("index", getRuntimeContext().getIndexOfThisSubtask(), false);
-
- isRunning = true;
+ if (index == 0) {
+ index = getRuntimeContext().getIndexOfThisSubtask();
+ }
}
@Override
public void run(SourceContext<String> ctx) throws Exception {
final Object lockingObject = ctx.getCheckpointLock();
- while (isRunning && index.value() < numElements) {
- char first = (char) ((index.value() % 40) + 40);
+ while (isRunning && index < numElements) {
+ char first = (char) ((index % 40) + 40);
stringBuilder.setLength(0);
stringBuilder.append(first);
@@ -166,7 +162,7 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
String result = randomString(stringBuilder, rnd);
synchronized (lockingObject) {
- index.update(index.value() + step);
+ index += step;
ctx.collect(result);
}
}
@@ -187,6 +183,16 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
return bld.toString();
}
+
+ @Override
+ public Integer snapshotState(long checkpointId, long checkpointTimestamp) {
+ return index;
+ }
+
+ @Override
+ public void restoreState(Integer state) {
+ index = state;
+ }
}
private static class StatefulCounterFunction extends RichMapFunction<PrefixCount, PrefixCount> implements Checkpointed<Long> {
@@ -215,8 +221,12 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
count = state;
}
}
-
- private static class OnceFailingPrefixCounter extends RichMapFunction<PrefixCount, PrefixCount> {
+
+ /**
+ * This function uses simultaneously the key/value state and is checkpointed.
+ */
+ private static class OnceFailingPrefixCounter extends RichMapFunction<PrefixCount, PrefixCount>
+ implements Checkpointed<Long> {
private static Map<String, Long> prefixCounts = new ConcurrentHashMap<String, Long>();
static final long[] counts = new long[PARALLELISM];
@@ -229,7 +239,7 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
private long count;
private OperatorState<Long> pCount;
- private OperatorState<Long> inputCount;
+ private long inputCount;
OnceFailingPrefixCounter(long numElements) {
this.numElements = numElements;
@@ -242,13 +252,12 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
failurePos = (new Random().nextLong() % (failurePosMax - failurePosMin)) + failurePosMin;
count = 0;
- pCount = getRuntimeContext().getOperatorState("prefix-count", 0L, true);
- inputCount = getRuntimeContext().getOperatorState("input-count", 0L, false);
+ pCount = getRuntimeContext().getKeyValueState(Long.class, 0L);
}
@Override
public void close() throws IOException {
- counts[getRuntimeContext().getIndexOfThisSubtask()] = inputCount.value();
+ counts[getRuntimeContext().getIndexOfThisSubtask()] = inputCount;
}
@Override
@@ -256,9 +265,9 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
count++;
if (!hasFailed && count >= failurePos) {
hasFailed = true;
- throw new Exception("Test Failure");
+// throw new Exception("Test Failure");
}
- inputCount.update(inputCount.value() + 1);
+ inputCount++;
long currentPrefixCount = pCount.value() + value.count;
pCount.update(currentPrefixCount);
@@ -266,12 +275,23 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
value.count = currentPrefixCount;
return value;
}
+
+ @Override
+ public Long snapshotState(long checkpointId, long checkpointTimestamp) {
+ return inputCount;
+ }
+
+ @Override
+ public void restoreState(Long state) {
+ inputCount = state;
+ }
}
private static class StringRichFilterFunction extends RichFilterFunction<String> implements Checkpointed<Long> {
-
- Long count = 0L;
+
static final long[] counts = new long[PARALLELISM];
+
+ private long count;
@Override
public boolean filter(String value) {
@@ -285,7 +305,7 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
}
@Override
- public Long snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
+ public Long snapshotState(long checkpointId, long checkpointTimestamp) {
return count;
}
@@ -296,49 +316,31 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
}
private static class StringPrefixCountRichMapFunction extends RichMapFunction<String, PrefixCount>
- implements Checkpointed<Integer> {
-
- OperatorState<Long> count;
+ implements Checkpointed<Long> {
+
static final long[] counts = new long[PARALLELISM];
+
+ private long count;
@Override
public PrefixCount map(String value) throws IOException {
- count.update(count.value() + 1);
+ count++;
return new PrefixCount(value.substring(0, 1), value, 1L);
}
-
- @Override
- public void open(Configuration conf) throws IOException {
- this.count = getRuntimeContext().getOperatorState("count", 0L, false,
- new StateCheckpointer<Long, String>() {
-
- @Override
- public String snapshotState(Long state, long id, long ts) {
- return state.toString();
- }
-
- @Override
- public Long restoreState(String stateSnapshot) {
- return Long.parseLong(stateSnapshot);
- }
-
- });
- }
@Override
public void close() throws IOException {
- counts[getRuntimeContext().getIndexOfThisSubtask()] = count.value();
+ counts[getRuntimeContext().getIndexOfThisSubtask()] = count;
}
@Override
- public Integer snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
- return null;
+ public Long snapshotState(long checkpointId, long checkpointTimestamp) {
+ return count;
}
@Override
- public void restoreState(Integer state) {
- // verify that we never store/restore null state
- fail();
+ public void restoreState(Long state) {
+ count = state;
}
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UdfStreamOperatorCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UdfStreamOperatorCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UdfStreamOperatorCheckpointingITCase.java
index cb02d2f..c12bcb9 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UdfStreamOperatorCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UdfStreamOperatorCheckpointingITCase.java
@@ -22,10 +22,10 @@ import com.google.common.collect.EvictingQueue;
import org.apache.flink.api.common.functions.FoldFunction;
import org.apache.flink.api.common.functions.ReduceFunction;
import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.common.state.OperatorState;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.checkpoint.Checkpointed;
import org.apache.flink.streaming.api.datastream.KeyedStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.sink.SinkFunction;
@@ -142,32 +142,36 @@ public class UdfStreamOperatorCheckpointingITCase extends StreamFaultToleranceTe
* Produces a sequence multiple times for each parallelism instance of downstream operators,
* augmented by the designated parallel subtaskId. The source is not parallel to ensure order.
*/
- private static class StatefulMultipleSequence extends RichSourceFunction<Tuple2<Integer, Long>>{
+ private static class StatefulMultipleSequence extends RichSourceFunction<Tuple2<Integer, Long>>
+ implements Checkpointed<Long> {
- private transient OperatorState<Long> count;
-
- @Override
- public void open(Configuration parameters) throws Exception {
- super.open(parameters);
- count = getRuntimeContext().getOperatorState("count", 0L, false);
- }
+ private long count;
@Override
public void run(SourceContext<Tuple2<Integer, Long>> ctx) throws Exception {
Object lock = ctx.getCheckpointLock();
- while (count.value() < NUM_INPUT){
+ while (count < NUM_INPUT){
synchronized (lock){
for (int i = 0; i < PARALLELISM; i++) {
- ctx.collect(Tuple2.of(i, count.value() + 1));
+ ctx.collect(Tuple2.of(i, count + 1));
}
- count.update(count.value() + 1);
+ count++;
}
}
}
@Override
- public void cancel() {
+ public void cancel() {}
+
+ @Override
+ public Long snapshotState(long checkpointId, long checkpointTimestamp) {
+ return count;
+ }
+
+ @Override
+ public void restoreState(Long state) {
+ count = state;
}
}
@@ -175,14 +179,15 @@ public class UdfStreamOperatorCheckpointingITCase extends StreamFaultToleranceTe
* Mapper that causes one failure between seeing 40% to 70% of the records.
*/
private static class OnceFailingIdentityMapFunction
- extends RichMapFunction<Tuple2<Integer, Long>, Tuple2<Integer, Long>> {
+ extends RichMapFunction<Tuple2<Integer, Long>, Tuple2<Integer, Long>>
+ implements Checkpointed<Long> {
private static volatile boolean hasFailed = false;
private final long numElements;
private long failurePos;
- private OperatorState<Long> count;
+ private long count;
public OnceFailingIdentityMapFunction(long numElements) {
this.numElements = numElements;
@@ -194,19 +199,28 @@ public class UdfStreamOperatorCheckpointingITCase extends StreamFaultToleranceTe
long failurePosMax = (long) (0.7 * numElements / getRuntimeContext().getNumberOfParallelSubtasks());
failurePos = (new Random().nextLong() % (failurePosMax - failurePosMin)) + failurePosMin;
- count = getRuntimeContext().getOperatorState("count", 0L, false);
}
@Override
public Tuple2<Integer, Long> map(Tuple2<Integer, Long> value) throws Exception {
- if (!hasFailed && count.value() >= failurePos) {
+ if (!hasFailed && count >= failurePos) {
hasFailed = true;
throw new Exception("Test Failure");
}
- count.update(count.value() + 1);
+ count++;
return value;
}
+
+ @Override
+ public Long snapshotState(long checkpointId, long checkpointTimestamp) {
+ return count;
+ }
+
+ @Override
+ public void restoreState(Long state) {
+ count = state;
+ }
}
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java
index dab6a6d..989db14 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/ClassLoaderITCase.java
@@ -23,6 +23,7 @@ import java.io.File;
import org.apache.flink.client.program.PackagedProgram;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.state.filesystem.FsStateBackendFactory;
import org.apache.flink.test.testdata.KMeansData;
import org.apache.flink.test.util.ForkableFlinkMiniCluster;
import org.junit.Rule;
@@ -55,7 +56,8 @@ public class ClassLoaderITCase {
// we need to use the "filesystem" state backend to ensure FLINK-2543 is not happening again.
config.setString(ConfigConstants.STATE_BACKEND, "filesystem");
- config.setString(ConfigConstants.STATE_BACKEND_FS_DIR, "file://" + folder.newFolder().getAbsolutePath());
+ config.setString(FsStateBackendFactory.CHECKPOINT_DIRECTORY_URI_CONF_KEY,
+ folder.newFolder().getAbsoluteFile().toURI().toString());
ForkableFlinkMiniCluster testCluster = new ForkableFlinkMiniCluster(config, false);
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureStreamingRecoveryITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureStreamingRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureStreamingRecoveryITCase.java
index e7b1668..054b321 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureStreamingRecoveryITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/ProcessFailureStreamingRecoveryITCase.java
@@ -18,9 +18,6 @@
package org.apache.flink.test.recovery;
-
-import static org.junit.Assert.assertTrue;
-
import java.io.File;
import java.io.IOException;
import java.util.UUID;
@@ -29,17 +26,19 @@ import org.apache.commons.io.FileUtils;
import org.apache.flink.api.common.functions.MapFunction;
import org.apache.flink.api.common.functions.RichMapFunction;
import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.api.common.state.OperatorState;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.state.FileStateHandle;
import org.apache.flink.streaming.api.checkpoint.Checkpointed;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
+import org.apache.flink.streaming.api.state.filesystem.FsStateBackend;
+
import org.junit.Assert;
+import static org.junit.Assert.assertTrue;
+
/**
* Test for streaming program behaviour in case of TaskManager failure
* based on {@link AbstractProcessFailureRecoveryTest}.
@@ -72,7 +71,8 @@ public class ProcessFailureStreamingRecoveryITCase extends AbstractProcessFailur
env.getConfig().disableSysoutLogging();
env.setNumberOfExecutionRetries(1);
env.enableCheckpointing(200);
- env.setStateHandleProvider(FileStateHandle.createProvider(tempCheckpointDir.getAbsolutePath()));
+
+ env.setStateBackend(new FsStateBackend(tempCheckpointDir.getAbsoluteFile().toURI()));
DataStream<Long> result = env.addSource(new SleepyDurableGenerateSequence(coordinateDir, DATA_COUNT))
// add a non-chained no-op map to test the chain state restore logic
@@ -104,7 +104,8 @@ public class ProcessFailureStreamingRecoveryITCase extends AbstractProcessFailur
}
}
- public static class SleepyDurableGenerateSequence extends RichParallelSourceFunction<Long> {
+ public static class SleepyDurableGenerateSequence extends RichParallelSourceFunction<Long>
+ implements Checkpointed<Long> {
private static final long SLEEP_TIME = 50;
@@ -113,7 +114,7 @@ public class ProcessFailureStreamingRecoveryITCase extends AbstractProcessFailur
private volatile boolean isRunning = true;
- private OperatorState<Long> collected;
+ private long collected;
public SleepyDurableGenerateSequence(File coordinateDir, long end) {
this.coordinateDir = coordinateDir;
@@ -133,7 +134,7 @@ public class ProcessFailureStreamingRecoveryITCase extends AbstractProcessFailur
final File proceedFile = new File(coordinateDir, PROCEED_MARKER_FILE);
boolean checkForProceedFile = true;
- while (isRunning && collected.value() < toCollect) {
+ while (isRunning && collected < toCollect) {
// check if the proceed file exists (then we go full speed)
// if not, we always recheck and sleep
if (checkForProceedFile) {
@@ -146,21 +147,26 @@ public class ProcessFailureStreamingRecoveryITCase extends AbstractProcessFailur
}
synchronized (checkpointLock) {
- sourceCtx.collect(collected.value() * stepSize + congruence);
- collected.update(collected.value() + 1);
+ sourceCtx.collect(collected * stepSize + congruence);
+ collected++;
}
}
}
-
- @Override
- public void open(Configuration conf) throws IOException {
- collected = getRuntimeContext().getOperatorState("count", 0L, false);
- }
@Override
public void cancel() {
isRunning = false;
}
+
+ @Override
+ public Long snapshotState(long checkpointId, long checkpointTimestamp) {
+ return collected;
+ }
+
+ @Override
+ public void restoreState(Long state) {
+ collected = state;
+ }
}
public static class Mapper extends RichMapFunction<Long, Long> {
[07/24] flink git commit: [FLINK-2808] [streaming] Refactor and
extend state backend abstraction
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StreamOperatorState.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StreamOperatorState.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StreamOperatorState.java
deleted file mode 100644
index c33b94e..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StreamOperatorState.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/*
- * 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.state;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.Map;
-
-import org.apache.flink.api.common.state.OperatorState;
-import org.apache.flink.api.common.state.StateCheckpointer;
-import org.apache.flink.runtime.state.StateHandle;
-import org.apache.flink.runtime.state.StateHandleProvider;
-
-import com.google.common.collect.ImmutableMap;
-
-/**
- * Implementation of the {@link OperatorState} interface for non-partitioned
- * user states. It provides methods for checkpointing and restoring operator
- * states upon failure using the provided {@link StateCheckpointer} and
- * {@link StateHandleProvider}.
- *
- * @param <S>
- * Type of the underlying {@link OperatorState}.
- * @param <C>
- * Type of the state snapshot.
- */
-public class StreamOperatorState<S, C extends Serializable> implements OperatorState<S> {
-
- private S state;
- protected StateCheckpointer<S, C> checkpointer;
- protected final StateHandleProvider<Serializable> provider;
-
- private boolean restored = true;
- private Serializable checkpoint = null;
-
- @SuppressWarnings("unchecked")
- public StreamOperatorState(StateCheckpointer<S, C> checkpointer, StateHandleProvider<C> provider) {
- this.checkpointer = checkpointer;
- this.provider = (StateHandleProvider<Serializable>) provider;
- }
-
- @SuppressWarnings("unchecked")
- public StreamOperatorState(StateHandleProvider<C> provider) {
- this((StateCheckpointer<S, C>) new BasicCheckpointer(), provider);
- }
-
- @Override
- public S value() throws IOException {
- if (!restored) {
- // If the state is not restore it yet, restore at this point
- restoreWithCheckpointer();
- }
- return state;
- }
-
- @Override
- public void update(S state) throws IOException {
- if (state == null) {
- throw new RuntimeException("Cannot set state to null.");
- }
- if (!restored) {
- // If the value is updated before the restore it is overwritten
- restored = true;
- checkpoint = false;
- }
- this.state = state;
- }
-
- public void setDefaultState(S defaultState) throws IOException {
- if (value() == null) {
- update(defaultState);
- }
- }
-
- public StateCheckpointer<S, C> getCheckpointer() {
- return checkpointer;
- }
-
- public void setCheckpointer(StateCheckpointer<S, C> checkpointer) {
- this.checkpointer = checkpointer;
- }
-
- protected StateHandleProvider<Serializable> getStateHandleProvider() {
- return provider;
- }
-
- public StateHandle<Serializable> snapshotState(long checkpointId, long checkpointTimestamp)
- throws Exception {
- // If the state is restored we take a snapshot, otherwise return the last checkpoint
- return provider.createStateHandle(restored ? checkpointer.snapshotState(value(), checkpointId,
- checkpointTimestamp) : checkpoint);
- }
-
- public void restoreState(StateHandle<Serializable> snapshot, ClassLoader userCodeClassLoader) throws Exception {
- // We set the checkpoint for lazy restore
- checkpoint = snapshot.getState(userCodeClassLoader);
- restored = false;
- }
-
- @SuppressWarnings("unchecked")
- private void restoreWithCheckpointer() throws IOException {
- update(checkpointer.restoreState((C) checkpoint));
- restored = true;
- checkpoint = null;
- }
-
- public Map<Serializable, S> getPartitionedState() throws Exception {
- return ImmutableMap.of((Serializable) 0, state);
- }
-
- @Override
- public String toString() {
- return state.toString();
- }
-
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StreamStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StreamStateHandle.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StreamStateHandle.java
new file mode 100644
index 0000000..0fa5952
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StreamStateHandle.java
@@ -0,0 +1,28 @@
+/*
+ * 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.state;
+
+import org.apache.flink.runtime.state.StateHandle;
+
+import java.io.InputStream;
+
+/**
+ * A state handle that produces an input stream when resolved.
+ */
+public interface StreamStateHandle extends StateHandle<InputStream> {}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/WrapperStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/WrapperStateHandle.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/WrapperStateHandle.java
deleted file mode 100644
index 9105fd2..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/WrapperStateHandle.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * 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.state;
-
-import java.io.Serializable;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.runtime.state.LocalStateHandle;
-import org.apache.flink.runtime.state.StateHandle;
-
-/**
- * StateHandle that wraps the StateHandles for the operator states of chained
- * tasks. This is needed so the wrapped handles are properly discarded.
- *
- */
-public class WrapperStateHandle extends LocalStateHandle<Serializable> {
-
- private static final long serialVersionUID = 1L;
-
- public WrapperStateHandle(List<Tuple2<StateHandle<Serializable>, Map<String, OperatorStateHandle>>> state) {
- super((Serializable) state);
- }
-
- @Override
- public void discardState() throws Exception {
- @SuppressWarnings("unchecked")
- List<Tuple2<StateHandle<Serializable>, Map<String, OperatorStateHandle>>> chainedStates =
- (List<Tuple2<StateHandle<Serializable>, Map<String, OperatorStateHandle>>>) getState(null); // we can pass "null" here because the LocalStateHandle is not using the ClassLoader anyways
- for (Tuple2<StateHandle<Serializable>, Map<String, OperatorStateHandle>> state : chainedStates) {
- if (state != null) {
- if (state.f0 != null) {
- state.f0.discardState();
- }
- if (state.f1 != null) {
- for (StateHandle<Serializable> opState : state.f1.values()) {
- opState.discardState();
- }
- }
- }
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/AbstractFileState.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/AbstractFileState.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/AbstractFileState.java
new file mode 100644
index 0000000..c4a376e
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/AbstractFileState.java
@@ -0,0 +1,83 @@
+/*
+ * 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.state.filesystem;
+
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+
+import java.io.IOException;
+
+/**
+ * Base class for state that is stored in a file.
+ */
+public abstract class AbstractFileState implements java.io.Serializable {
+
+ private static final long serialVersionUID = 350284443258002355L;
+
+ /** The path to the file in the filesystem, fully describing the file system */
+ private final Path filePath;
+
+ /** Cached file system handle */
+ private transient FileSystem fs;
+
+ /**
+ * Creates a new file state for the given file path.
+ *
+ * @param filePath The path to the file that stores the state.
+ */
+ protected AbstractFileState(Path filePath) {
+ this.filePath = filePath;
+ }
+
+ /**
+ * Gets the path where this handle's state is stored.
+ * @return The path where this handle's state is stored.
+ */
+ public Path getFilePath() {
+ return filePath;
+ }
+
+ /**
+ * Discard the state by deleting the file that stores the state. If the parent directory
+ * of the state is empty after deleting the state file, it is also deleted.
+ *
+ * @throws Exception Thrown, if the file deletion (not the directory deletion) fails.
+ */
+ public void discardState() throws Exception {
+ getFileSystem().delete(filePath, false);
+
+ // send a call to delete the directory containing the file. this will
+ // fail (and be ignored) when some files still exist
+ try {
+ getFileSystem().delete(filePath.getParent(), false);
+ } catch (IOException ignored) {}
+ }
+
+ /**
+ * Gets the file system that stores the file state.
+ * @return The file system that stores the file state.
+ * @throws IOException Thrown if the file system cannot be accessed.
+ */
+ protected FileSystem getFileSystem() throws IOException {
+ if (fs == null) {
+ fs = FileSystem.get(filePath.toUri());
+ }
+ return fs;
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FileSerializableStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FileSerializableStateHandle.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FileSerializableStateHandle.java
new file mode 100644
index 0000000..9bf5ec1
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FileSerializableStateHandle.java
@@ -0,0 +1,53 @@
+/*
+ * 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.state.filesystem;
+
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.state.StateHandle;
+import org.apache.flink.util.InstantiationUtil;
+
+import java.io.ObjectInputStream;
+
+/**
+ * A state handle that points to state stored in a file via Java Serialization.
+ *
+ * @param <T> The type of state pointed to by the state handle.
+ */
+public class FileSerializableStateHandle<T> extends AbstractFileState implements StateHandle<T> {
+
+ private static final long serialVersionUID = -657631394290213622L;
+
+ /**
+ * Creates a new FileSerializableStateHandle pointing to state at the given file path.
+ *
+ * @param filePath The path to the file containing the checkpointed state.
+ */
+ public FileSerializableStateHandle(Path filePath) {
+ super(filePath);
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public T getState(ClassLoader classLoader) throws Exception {
+ FSDataInputStream inStream = getFileSystem().open(getFilePath());
+ ObjectInputStream ois = new InstantiationUtil.ClassLoaderObjectInputStream(inStream, classLoader);
+ return (T) ois.readObject();
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FileStreamStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FileStreamStateHandle.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FileStreamStateHandle.java
new file mode 100644
index 0000000..79512d7
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FileStreamStateHandle.java
@@ -0,0 +1,46 @@
+/*
+ * 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.state.filesystem;
+
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.streaming.api.state.StreamStateHandle;
+
+import java.io.InputStream;
+
+/**
+ * A state handle that points to state in a file system, accessible as an input stream.
+ */
+public class FileStreamStateHandle extends AbstractFileState implements StreamStateHandle {
+
+ private static final long serialVersionUID = -6826990484549987311L;
+
+ /**
+ * Creates a new FileStreamStateHandle pointing to state at the given file path.
+ *
+ * @param filePath The path to the file containing the checkpointed state.
+ */
+ public FileStreamStateHandle(Path filePath) {
+ super(filePath);
+ }
+
+ @Override
+ public InputStream getState(ClassLoader userCodeClassLoader) throws Exception {
+ return getFileSystem().open(getFilePath());
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsHeapKvState.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsHeapKvState.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsHeapKvState.java
new file mode 100644
index 0000000..107a3be
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsHeapKvState.java
@@ -0,0 +1,88 @@
+/*
+ * 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.state.filesystem;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.OutputViewDataOutputStreamWrapper;
+import org.apache.flink.streaming.api.state.AbstractHeapKvState;
+
+import java.io.DataOutputStream;
+import java.util.HashMap;
+
+/**
+ * Heap-backed key/value state that is snapshotted into files.
+ *
+ * @param <K> The type of the key.
+ * @param <V> The type of the value.
+ */
+public class FsHeapKvState<K, V> extends AbstractHeapKvState<K, V, FsStateBackend> {
+
+ /** The file system state backend backing snapshots of this state */
+ private final FsStateBackend backend;
+
+ /**
+ * Creates a new and empty key/value state.
+ *
+ * @param keySerializer The serializer for the key.
+ * @param valueSerializer The serializer for the value.
+ * @param defaultValue The value that is returned when no other value has been associated with a key, yet.
+ * @param backend The file system state backend backing snapshots of this state
+ */
+ public FsHeapKvState(TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer,
+ V defaultValue, FsStateBackend backend) {
+ super(keySerializer, valueSerializer, defaultValue);
+ this.backend = backend;
+ }
+
+ /**
+ * Creates a new key/value state with the given state contents.
+ * This method is used to re-create key/value state with existing data, for example from
+ * a snapshot.
+ *
+ * @param keySerializer The serializer for the key.
+ * @param valueSerializer The serializer for the value.
+ * @param defaultValue The value that is returned when no other value has been associated with a key, yet.
+ * @param state The map of key/value pairs to initialize the state with.
+ * @param backend The file system state backend backing snapshots of this state
+ */
+ public FsHeapKvState(TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer,
+ V defaultValue, HashMap<K, V> state, FsStateBackend backend) {
+ super(keySerializer, valueSerializer, defaultValue, state);
+ this.backend = backend;
+ }
+
+
+ @Override
+ public FsHeapKvStateSnapshot<K, V> shapshot(long checkpointId, long timestamp) throws Exception {
+ // first, create an output stream to write to
+ try (FsStateBackend.FsCheckpointStateOutputStream out =
+ backend.createCheckpointStateOutputStream(checkpointId, timestamp)) {
+
+ // serialize the state to the output stream
+ OutputViewDataOutputStreamWrapper outView =
+ new OutputViewDataOutputStreamWrapper(new DataOutputStream(out));
+ outView.writeInt(size());
+ writeStateToOutputView(outView);
+ outView.flush();
+
+ // create a handle to the state
+ return new FsHeapKvStateSnapshot<>(getKeySerializer(), getValueSerializer(), out.closeAndGetPath());
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsHeapKvStateSnapshot.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsHeapKvStateSnapshot.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsHeapKvStateSnapshot.java
new file mode 100644
index 0000000..c7117f8
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsHeapKvStateSnapshot.java
@@ -0,0 +1,95 @@
+/*
+ * 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.state.filesystem;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.fs.FSDataInputStream;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.memory.InputViewDataInputStreamWrapper;
+import org.apache.flink.streaming.api.state.KvStateSnapshot;
+
+import java.io.DataInputStream;
+import java.util.HashMap;
+
+/**
+ * A snapshot of a heap key/value state stored in a file.
+ *
+ * @param <K> The type of the key in the snapshot state.
+ * @param <V> The type of the value in the snapshot state.
+ */
+public class FsHeapKvStateSnapshot<K, V> extends AbstractFileState implements KvStateSnapshot<K, V, FsStateBackend> {
+
+ private static final long serialVersionUID = 1L;
+
+ /** Name of the key serializer class */
+ private final String keySerializerClassName;
+
+ /** Name of the value serializer class */
+ private final String valueSerializerClassName;
+
+ /**
+ * Creates a new state snapshot with data in the file system.
+ *
+ * @param keySerializer The serializer for the keys.
+ * @param valueSerializer The serializer for the values.
+ * @param filePath The path where the snapshot data is stored.
+ */
+ public FsHeapKvStateSnapshot(TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer, Path filePath) {
+ super(filePath);
+ this.keySerializerClassName = keySerializer.getClass().getName();
+ this.valueSerializerClassName = valueSerializer.getClass().getName();
+ }
+
+ @Override
+ public FsHeapKvState<K, V> restoreState(
+ FsStateBackend stateBackend,
+ final TypeSerializer<K> keySerializer,
+ final TypeSerializer<V> valueSerializer,
+ V defaultValue,
+ ClassLoader classLoader) throws Exception {
+
+ // validity checks
+ if (!keySerializer.getClass().getName().equals(keySerializerClassName) ||
+ !valueSerializer.getClass().getName().equals(valueSerializerClassName)) {
+ throw new IllegalArgumentException(
+ "Cannot restore the state from the snapshot with the given serializers. " +
+ "State (K/V) was serialized with (" + valueSerializerClassName +
+ "/" + keySerializerClassName + ")");
+ }
+
+ // state restore
+ try (FSDataInputStream inStream = stateBackend.getFileSystem().open(getFilePath())) {
+ InputViewDataInputStreamWrapper inView = new InputViewDataInputStreamWrapper(new DataInputStream(inStream));
+
+ final int numEntries = inView.readInt();
+ HashMap<K, V> stateMap = new HashMap<>(numEntries);
+
+ for (int i = 0; i < numEntries; i++) {
+ K key = keySerializer.deserialize(inView);
+ V value = valueSerializer.deserialize(inView);
+ stateMap.put(key, value);
+ }
+
+ return new FsHeapKvState<K, V>(keySerializer, valueSerializer, defaultValue, stateMap, stateBackend);
+ }
+ catch (Exception e) {
+ throw new Exception("Failed to restore state from file system", e);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsStateBackend.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsStateBackend.java
new file mode 100644
index 0000000..1fc2457
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsStateBackend.java
@@ -0,0 +1,409 @@
+/*
+ * 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.state.filesystem;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.fs.FSDataOutputStream;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.runtime.state.StateHandle;
+import org.apache.flink.streaming.api.state.StateBackend;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.UUID;
+
+/**
+ * The file state backend is a state backend that stores the state of streaming jobs in a file system.
+ *
+ * <p>The state backend has one core directory into which it puts all checkpoint data. Inside that
+ * directory, it creates a directory per job, inside which each checkpoint gets a directory, with
+ * files for each state, for example:
+ *
+ * {@code hdfs://namenode:port/flink-checkpoints/<job-id>/chk-17/6ba7b810-9dad-11d1-80b4-00c04fd430c8 }
+ */
+public class FsStateBackend extends StateBackend<FsStateBackend> {
+
+ private static final long serialVersionUID = -8191916350224044011L;
+
+ private static final Logger LOG = LoggerFactory.getLogger(FsStateBackend.class);
+
+
+ /** The path to the directory for the checkpoint data, including the file system
+ * description via scheme and optional authority */
+ private final Path basePath;
+
+ /** The directory (job specific) into this initialized instance of the backend stores its data */
+ private transient Path checkpointDirectory;
+
+ /** Cached handle to the file system for file operations */
+ private transient FileSystem filesystem;
+
+
+ /**
+ * Creates a new state backend that stores its checkpoint data in the file system and location
+ * defined by the given URI.
+ *
+ * <p>A file system for the file system scheme in the URI (e.g., 'file://', 'hdfs://', or 'S3://')
+ * must be accessible via {@link FileSystem#get(URI)}.
+ *
+ * <p>For a state backend targeting HDFS, this means that the URI must either specify the authority
+ * (host and port), or that the Hadoop configuration that describes that information must be in the
+ * classpath.
+ *
+ * @param checkpointDataUri The URI describing the filesystem (scheme and optionally authority),
+ * and the path to teh checkpoint data directory.
+ * @throws IOException Thrown, if no file system can be found for the scheme in the URI.
+ */
+ public FsStateBackend(String checkpointDataUri) throws IOException {
+ this(new Path(checkpointDataUri));
+ }
+
+ /**
+ * Creates a new state backend that stores its checkpoint data in the file system and location
+ * defined by the given URI.
+ *
+ * <p>A file system for the file system scheme in the URI (e.g., 'file://', 'hdfs://', or 'S3://')
+ * must be accessible via {@link FileSystem#get(URI)}.
+ *
+ * <p>For a state backend targeting HDFS, this means that the URI must either specify the authority
+ * (host and port), or that the Hadoop configuration that describes that information must be in the
+ * classpath.
+ *
+ * @param checkpointDataUri The URI describing the filesystem (scheme and optionally authority),
+ * and the path to teh checkpoint data directory.
+ * @throws IOException Thrown, if no file system can be found for the scheme in the URI.
+ */
+ public FsStateBackend(Path checkpointDataUri) throws IOException {
+ this(checkpointDataUri.toUri());
+ }
+
+ /**
+ * Creates a new state backend that stores its checkpoint data in the file system and location
+ * defined by the given URI.
+ *
+ * <p>A file system for the file system scheme in the URI (e.g., 'file://', 'hdfs://', or 'S3://')
+ * must be accessible via {@link FileSystem#get(URI)}.
+ *
+ * <p>For a state backend targeting HDFS, this means that the URI must either specify the authority
+ * (host and port), or that the Hadoop configuration that describes that information must be in the
+ * classpath.
+ *
+ * @param checkpointDataUri The URI describing the filesystem (scheme and optionally authority),
+ * and the path to teh checkpoint data directory.
+ * @throws IOException Thrown, if no file system can be found for the scheme in the URI.
+ */
+ public FsStateBackend(URI checkpointDataUri) throws IOException {
+ final String scheme = checkpointDataUri.getScheme();
+ final String path = checkpointDataUri.getPath();
+
+ // some validity checks
+ if (scheme == null) {
+ throw new IllegalArgumentException("The scheme (hdfs://, file://, etc) is null. " +
+ "Please specify the file system scheme explicitly in the URI.");
+ }
+ if (path == null) {
+ throw new IllegalArgumentException("The path to store the checkpoint data in is null. " +
+ "Please specify a directory path for the checkpoint data.");
+ }
+ if (path.length() == 0 || path.equals("/")) {
+ throw new IllegalArgumentException("Cannot use the root directory for checkpoints.");
+ }
+
+ // we do a bit of work to make sure that the URI for the filesystem refers to exactly the same
+ // (distributed) filesystem on all hosts and includes full host/port information, even if the
+ // original URI did not include that. We count on the filesystem loading from the configuration
+ // to fill in the missing data.
+
+ // try to grab the file system for this path/URI
+ this.filesystem = FileSystem.get(checkpointDataUri);
+ if (this.filesystem == null) {
+ throw new IOException("Could not find a file system for the given scheme in the available configurations.");
+ }
+
+ URI fsURI = this.filesystem.getUri();
+ try {
+ URI baseURI = new URI(fsURI.getScheme(), fsURI.getAuthority(), path, null, null);
+ this.basePath = new Path(baseURI);
+ }
+ catch (URISyntaxException e) {
+ throw new IOException(
+ String.format("Cannot create file system URI for checkpointDataUri %s and filesystem URI %s",
+ checkpointDataUri, fsURI), e);
+ }
+ }
+
+ /**
+ * Gets the base directory where all state-containing files are stored.
+ * The job specific directory is created inside this directory.
+ *
+ * @return The base directory.
+ */
+ public Path getBasePath() {
+ return basePath;
+ }
+
+ /**
+ * Gets the directory where this state backend stores its checkpoint data. Will be null if
+ * the state backend has not been initialized.
+ *
+ * @return The directory where this state backend stores its checkpoint data.
+ */
+ public Path getCheckpointDirectory() {
+ return checkpointDirectory;
+ }
+
+ /**
+ * Checks whether this state backend is initialized. Note that initialization does not carry
+ * across serialization. After each serialization, the state backend needs to be initialized.
+ *
+ * @return True, if the file state backend has been initialized, false otherwise.
+ */
+ public boolean isInitialized() {
+ return filesystem != null && checkpointDirectory != null;
+ }
+
+ /**
+ * Gets the file system handle for the file system that stores the state for this backend.
+ *
+ * @return This backend's file system handle.
+ */
+ public FileSystem getFileSystem() {
+ if (filesystem != null) {
+ return filesystem;
+ }
+ else {
+ throw new IllegalStateException("State backend has not been initialized.");
+ }
+ }
+
+ // ------------------------------------------------------------------------
+ // initialization and cleanup
+ // ------------------------------------------------------------------------
+
+ @Override
+ public void initializeForJob(JobID jobId) throws Exception {
+ Path dir = new Path(basePath, jobId.toString());
+
+ LOG.info("Initializing file state backend to URI " + dir);
+
+ filesystem = basePath.getFileSystem();
+ filesystem.mkdirs(dir);
+
+ checkpointDirectory = dir;
+ }
+
+ @Override
+ public void disposeAllStateForCurrentJob() throws Exception {
+ FileSystem fs = this.filesystem;
+ Path dir = this.checkpointDirectory;
+
+ if (fs != null && dir != null) {
+ this.filesystem = null;
+ this.checkpointDirectory = null;
+ fs.delete(dir, true);
+ }
+ else {
+ throw new IllegalStateException("state backend has not been initialized");
+ }
+ }
+
+ // ------------------------------------------------------------------------
+ // state backend operations
+ // ------------------------------------------------------------------------
+
+ @Override
+ public <K, V> FsHeapKvState<K, V> createKvState(
+ TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer, V defaultValue) throws Exception {
+ return new FsHeapKvState<K, V>(keySerializer, valueSerializer, defaultValue, this);
+ }
+
+ @Override
+ public <S extends Serializable> StateHandle<S> checkpointStateSerializable(
+ S state, long checkpointID, long timestamp) throws Exception
+ {
+ checkFileSystemInitialized();
+
+ // make sure the directory for that specific checkpoint exists
+ final Path checkpointDir = createCheckpointDirPath(checkpointID);
+ filesystem.mkdirs(checkpointDir);
+
+
+ Exception latestException = null;
+
+ for (int attempt = 0; attempt < 10; attempt++) {
+ Path targetPath = new Path(checkpointDir, UUID.randomUUID().toString());
+ FSDataOutputStream outStream;
+ try {
+ outStream = filesystem.create(targetPath, false);
+ }
+ catch (Exception e) {
+ latestException = e;
+ continue;
+ }
+
+ ObjectOutputStream os = new ObjectOutputStream(outStream);
+ os.writeObject(state);
+ os.close();
+ return new FileSerializableStateHandle<S>(targetPath);
+ }
+
+ throw new Exception("Could not open output stream for state backend", latestException);
+ }
+
+ @Override
+ public FsCheckpointStateOutputStream createCheckpointStateOutputStream(long checkpointID, long timestamp) throws Exception {
+ checkFileSystemInitialized();
+
+ final Path checkpointDir = createCheckpointDirPath(checkpointID);
+ filesystem.mkdirs(checkpointDir);
+
+ Exception latestException = null;
+
+ for (int attempt = 0; attempt < 10; attempt++) {
+ Path targetPath = new Path(checkpointDir, UUID.randomUUID().toString());
+ try {
+ FSDataOutputStream outStream = filesystem.create(targetPath, false);
+ return new FsCheckpointStateOutputStream(outStream, targetPath, filesystem);
+ }
+ catch (Exception e) {
+ latestException = e;
+ }
+ }
+ throw new Exception("Could not open output stream for state backend", latestException);
+ }
+
+ // ------------------------------------------------------------------------
+ // utilities
+ // ------------------------------------------------------------------------
+
+ private void checkFileSystemInitialized() throws IllegalStateException {
+ if (filesystem == null || checkpointDirectory == null) {
+ throw new IllegalStateException("filesystem has not been re-initialized after deserialization");
+ }
+ }
+
+ private Path createCheckpointDirPath(long checkpointID) {
+ return new Path(checkpointDirectory, "chk-" + checkpointID);
+ }
+
+ @Override
+ public String toString() {
+ return checkpointDirectory == null ?
+ "File State Backend @ " + basePath :
+ "File State Backend (initialized) @ " + checkpointDirectory;
+ }
+
+ // ------------------------------------------------------------------------
+ // Output stream for state checkpointing
+ // ------------------------------------------------------------------------
+
+ /**
+ * A CheckpointStateOutputStream that writes into a file and returns the path to that file upon
+ * closing.
+ */
+ public static final class FsCheckpointStateOutputStream extends CheckpointStateOutputStream {
+
+ private final FSDataOutputStream outStream;
+
+ private final Path filePath;
+
+ private final FileSystem fs;
+
+ private boolean closed;
+
+ FsCheckpointStateOutputStream(FSDataOutputStream outStream, Path filePath, FileSystem fs) {
+ this.outStream = outStream;
+ this.filePath = filePath;
+ this.fs = fs;
+ }
+
+
+ @Override
+ public void write(int b) throws IOException {
+ outStream.write(b);
+ }
+
+ @Override
+ public void write(byte[] b, int off, int len) throws IOException {
+ outStream.write(b, off, len);
+ }
+
+ @Override
+ public void flush() throws IOException {
+ outStream.flush();
+ }
+
+ /**
+ * If the stream is only closed, we remove the produced file (cleanup through the auto close
+ * feature, for example). This method throws no exception if the deletion fails, but only
+ * logs the error.
+ */
+ @Override
+ public void close() {
+ synchronized (this) {
+ if (!closed) {
+ closed = true;
+ try {
+ outStream.close();
+ fs.delete(filePath, false);
+
+ // attempt to delete the parent (will fail and be ignored if the parent has more files)
+ try {
+ fs.delete(filePath.getParent(), false);
+ } catch (IOException ignored) {}
+ }
+ catch (Exception e) {
+ LOG.warn("Cannot delete closed and discarded state stream to " + filePath, e);
+ }
+ }
+ }
+ }
+
+ @Override
+ public FileStreamStateHandle closeAndGetHandle() throws IOException {
+ return new FileStreamStateHandle(closeAndGetPath());
+ }
+
+ /**
+ * Closes the stream and returns the path to the file that contains the stream's data.
+ * @return The path to the file that contains the stream's data.
+ * @throws IOException Thrown if the stream cannot be successfully closed.
+ */
+ public Path closeAndGetPath() throws IOException {
+ synchronized (this) {
+ if (!closed) {
+ closed = true;
+ outStream.close();
+ return filePath;
+ }
+ else {
+ throw new IOException("Stream has already been closed and discarded.");
+ }
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsStateBackendFactory.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsStateBackendFactory.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsStateBackendFactory.java
new file mode 100644
index 0000000..f0ad6bd
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsStateBackendFactory.java
@@ -0,0 +1,56 @@
+/*
+ * 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.state.filesystem;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.IllegalConfigurationException;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.streaming.api.state.StateBackendFactory;
+
+/**
+ * A factory that creates an {@link org.apache.flink.streaming.api.state.filesystem.FsStateBackend}
+ * from a configuration.
+ */
+public class FsStateBackendFactory implements StateBackendFactory<FsStateBackend> {
+
+ /** The key under which the config stores the directory where checkpoints should be stored */
+ public static final String CHECKPOINT_DIRECTORY_URI_CONF_KEY = "state.backend.fs.checkpointdir";
+
+
+ @Override
+ public FsStateBackend createFromConfig(Configuration config) throws Exception {
+ String checkpointDirURI = config.getString(CHECKPOINT_DIRECTORY_URI_CONF_KEY, null);
+
+ if (checkpointDirURI == null) {
+ throw new IllegalConfigurationException(
+ "Cannot create the file system state backend: The configuration does not specify the " +
+ "checkpoint directory '" + CHECKPOINT_DIRECTORY_URI_CONF_KEY + '\'');
+ }
+
+ try {
+ Path path = new Path(checkpointDirURI);
+ return new FsStateBackend(path);
+ }
+ catch (IllegalArgumentException e) {
+ throw new Exception("Cannot initialize File System State Backend with URI '"
+ + checkpointDirURI + '.', e);
+ }
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/ByteStreamStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/ByteStreamStateHandle.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/ByteStreamStateHandle.java
new file mode 100644
index 0000000..7952e58
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/ByteStreamStateHandle.java
@@ -0,0 +1,52 @@
+/*
+ * 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.state.memory;
+
+import org.apache.flink.streaming.api.state.StreamStateHandle;
+
+import java.io.ByteArrayInputStream;
+import java.io.InputStream;
+
+/**
+ * A state handle that contains stream state in a byte array.
+ */
+public final class ByteStreamStateHandle implements StreamStateHandle {
+
+ private static final long serialVersionUID = -5280226231200217594L;
+
+ /** the state data */
+ private final byte[] data;
+
+ /**
+ * Creates a new ByteStreamStateHandle containing the given data.
+ *
+ * @param data The state data.
+ */
+ public ByteStreamStateHandle(byte[] data) {
+ this.data = data;
+ }
+
+ @Override
+ public InputStream getState(ClassLoader userCodeClassLoader) {
+ return new ByteArrayInputStream(data);
+ }
+
+ @Override
+ public void discardState() {}
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/MemHeapKvState.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/MemHeapKvState.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/MemHeapKvState.java
new file mode 100644
index 0000000..e611887
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/MemHeapKvState.java
@@ -0,0 +1,52 @@
+/*
+ * 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.state.memory;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.runtime.util.DataOutputSerializer;
+import org.apache.flink.streaming.api.state.AbstractHeapKvState;
+
+import java.util.HashMap;
+
+/**
+ * Heap-backed key/value state that is snapshotted into a serialized memory copy.
+ *
+ * @param <K> The type of the key.
+ * @param <V> The type of the value.
+ */
+public class MemHeapKvState<K, V> extends AbstractHeapKvState<K, V, MemoryStateBackend> {
+
+ public MemHeapKvState(TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer, V defaultValue) {
+ super(keySerializer, valueSerializer, defaultValue);
+ }
+
+ public MemHeapKvState(TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer,
+ V defaultValue, HashMap<K, V> state) {
+ super(keySerializer, valueSerializer, defaultValue, state);
+ }
+
+ @Override
+ public MemoryHeapKvStateSnapshot<K, V> shapshot(long checkpointId, long timestamp) throws Exception {
+ DataOutputSerializer ser = new DataOutputSerializer(Math.max(size() * 16, 16));
+ writeStateToOutputView(ser);
+ byte[] bytes = ser.getCopyOfBuffer();
+
+ return new MemoryHeapKvStateSnapshot<K, V>(getKeySerializer(), getValueSerializer(), bytes, size());
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/MemoryHeapKvStateSnapshot.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/MemoryHeapKvStateSnapshot.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/MemoryHeapKvStateSnapshot.java
new file mode 100644
index 0000000..7f50379
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/MemoryHeapKvStateSnapshot.java
@@ -0,0 +1,102 @@
+/*
+ * 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.state.memory;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.runtime.util.DataInputDeserializer;
+import org.apache.flink.streaming.api.state.KvStateSnapshot;
+
+import java.util.HashMap;
+
+/**
+ * A snapshot of a {@link MemHeapKvState} for a checkpoint. The data is stored in a heap byte
+ * array, in serialized form.
+ *
+ * @param <K> The type of the key in the snapshot state.
+ * @param <V> The type of the value in the snapshot state.
+ */
+public class MemoryHeapKvStateSnapshot<K, V> implements KvStateSnapshot<K, V, MemoryStateBackend> {
+
+ private static final long serialVersionUID = 1L;
+
+ /** Name of the key serializer class */
+ private final String keySerializerClassName;
+
+ /** Name of the value serializer class */
+ private final String valueSerializerClassName;
+
+ /** The serialized data of the state key/value pairs */
+ private final byte[] data;
+
+ /** The number of key/value pairs */
+ private final int numEntries;
+
+ /**
+ * Creates a new heap memory state snapshot.
+ *
+ * @param keySerializer The serializer for the keys.
+ * @param valueSerializer The serializer for the values.
+ * @param data The serialized data of the state key/value pairs
+ * @param numEntries The number of key/value pairs
+ */
+ public MemoryHeapKvStateSnapshot(TypeSerializer<K> keySerializer,
+ TypeSerializer<V> valueSerializer, byte[] data, int numEntries) {
+ this.keySerializerClassName = keySerializer.getClass().getName();
+ this.valueSerializerClassName = valueSerializer.getClass().getName();
+ this.data = data;
+ this.numEntries = numEntries;
+ }
+
+
+ @Override
+ public MemHeapKvState<K, V> restoreState(
+ MemoryStateBackend stateBackend,
+ final TypeSerializer<K> keySerializer,
+ final TypeSerializer<V> valueSerializer,
+ V defaultValue,
+ ClassLoader classLoader) throws Exception {
+
+ // validity checks
+ if (!keySerializer.getClass().getName().equals(keySerializerClassName) ||
+ !valueSerializer.getClass().getName().equals(valueSerializerClassName)) {
+ throw new IllegalArgumentException(
+ "Cannot restore the state from the snapshot with the given serializers. " +
+ "State (K/V) was serialized with (" + valueSerializerClassName +
+ "/" + keySerializerClassName + ")");
+ }
+
+ // restore state
+ HashMap<K, V> stateMap = new HashMap<>(numEntries);
+ DataInputDeserializer in = new DataInputDeserializer(data, 0, data.length);
+
+ for (int i = 0; i < numEntries; i++) {
+ K key = keySerializer.deserialize(in);
+ V value = valueSerializer.deserialize(in);
+ stateMap.put(key, value);
+ }
+
+ return new MemHeapKvState<K, V>(keySerializer, valueSerializer, defaultValue, stateMap);
+ }
+
+ /**
+ * Discarding the heap state is a no-op.
+ */
+ @Override
+ public void discardState() {}
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/MemoryStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/MemoryStateBackend.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/MemoryStateBackend.java
new file mode 100644
index 0000000..b2dfae8
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/MemoryStateBackend.java
@@ -0,0 +1,206 @@
+/*
+ * 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.state.memory;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.runtime.state.StateHandle;
+import org.apache.flink.streaming.api.state.StateBackend;
+import org.apache.flink.streaming.api.state.StreamStateHandle;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.Serializable;
+
+/**
+ * A {@link StateBackend} that stores all its data and checkpoints in memory and has no
+ * capabilities to spill to disk. Checkpoints are serialized and the serialized data is
+ * transferred
+ */
+public class MemoryStateBackend extends StateBackend<MemoryStateBackend> {
+
+ private static final long serialVersionUID = 4109305377809414635L;
+
+ /** The default maximal size that the snapshotted memory state may have (5 MiBytes) */
+ private static final int DEFAULT_MAX_STATE_SIZE = 5 * 1024 * 1024;
+
+ /** The maximal size that the snapshotted memory state may have */
+ private final int maxStateSize;
+
+ /**
+ * Creates a new memory state backend that accepts states whose serialized forms are
+ * up to the default state size (5 MB).
+ */
+ public MemoryStateBackend() {
+ this(DEFAULT_MAX_STATE_SIZE);
+ }
+
+ /**
+ * Creates a new memory state backend that accepts states whose serialized forms are
+ * up to the given number of bytes.
+ *
+ * @param maxStateSize The maximal size of the serialized state
+ */
+ public MemoryStateBackend(int maxStateSize) {
+ this.maxStateSize = maxStateSize;
+ }
+
+ // ------------------------------------------------------------------------
+ // initialization and cleanup
+ // ------------------------------------------------------------------------
+
+ @Override
+ public void initializeForJob(JobID job) {
+ // nothing to do here
+ }
+
+ @Override
+ public void disposeAllStateForCurrentJob() {
+ // nothing to do here, GC will do it
+ }
+
+ // ------------------------------------------------------------------------
+ // State backend operations
+ // ------------------------------------------------------------------------
+
+ @Override
+ public <K, V> MemHeapKvState<K, V> createKvState(
+ TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer, V defaultValue) {
+ return new MemHeapKvState<K, V>(keySerializer, valueSerializer, defaultValue);
+ }
+
+ /**
+ * Serialized the given state into bytes using Java serialization and creates a state handle that
+ * can re-create that state.
+ *
+ * @param state The state to checkpoint.
+ * @param checkpointID The ID of the checkpoint.
+ * @param timestamp The timestamp of the checkpoint.
+ * @param <S> The type of the state.
+ *
+ * @return A state handle that contains the given state serialized as bytes.
+ * @throws Exception Thrown, if the serialization fails.
+ */
+ @Override
+ public <S extends Serializable> StateHandle<S> checkpointStateSerializable(
+ S state, long checkpointID, long timestamp) throws Exception
+ {
+ SerializedStateHandle<S> handle = new SerializedStateHandle<>(state);
+ checkSize(handle.getSizeOfSerializedState(), maxStateSize);
+ return new SerializedStateHandle<S>(state);
+ }
+
+ @Override
+ public CheckpointStateOutputStream createCheckpointStateOutputStream(
+ long checkpointID, long timestamp) throws Exception
+ {
+ return new MemoryCheckpointOutputStream(maxStateSize);
+ }
+
+ // ------------------------------------------------------------------------
+ // Utilities
+ // ------------------------------------------------------------------------
+
+ @Override
+ public String toString() {
+ return "MemoryStateBackend (data in heap memory / checkpoints to JobManager)";
+ }
+
+ static void checkSize(int size, int maxSize) throws IOException {
+ if (size > maxSize) {
+ throw new IOException(
+ "Size of the state is larger than the maximum permitted memory-backed state. Size="
+ + size + " , maxSize=" + maxSize
+ + " . Consider using a different state backend, like the File System State backend.");
+ }
+ }
+
+ // ------------------------------------------------------------------------
+
+ /**
+ * A CheckpointStateOutputStream that writes into a byte array.
+ */
+ public static final class MemoryCheckpointOutputStream extends CheckpointStateOutputStream {
+
+ private final ByteArrayOutputStream os = new ByteArrayOutputStream();
+
+ private final int maxSize;
+
+ private boolean closed;
+
+ public MemoryCheckpointOutputStream(int maxSize) {
+ this.maxSize = maxSize;
+ }
+
+ @Override
+ public void write(int b) {
+ os.write(b);
+ }
+
+ @Override
+ public void write(byte[] b, int off, int len) {
+ os.write(b, off, len);
+ }
+
+ // --------------------------------------------------------------------
+
+ @Override
+ public void close() {
+ closed = true;
+ os.reset();
+ }
+
+ @Override
+ public StreamStateHandle closeAndGetHandle() throws IOException {
+ return new ByteStreamStateHandle(closeAndGetBytes());
+ }
+
+ /**
+ * Closes the stream and returns the byte array containing the stream's data.
+ * @return The byte array containing the stream's data.
+ * @throws IOException Thrown if the size of the data exceeds the maximal
+ */
+ public byte[] closeAndGetBytes() throws IOException {
+ if (!closed) {
+ checkSize(os.size(), maxSize);
+ byte[] bytes = os.toByteArray();
+ close();
+ return bytes;
+ }
+ else {
+ throw new IllegalStateException("stream has already been closed");
+ }
+ }
+ }
+
+ // ------------------------------------------------------------------------
+ // Static default instance
+ // ------------------------------------------------------------------------
+
+ /** The default instance of this state backend, using the default maximal state size */
+ private static final MemoryStateBackend DEFAULT_INSTANCE = new MemoryStateBackend();
+
+ /**
+ * Gets the default instance of this state backend, using the default maximal state size.
+ * @return The default instance of this state backend.
+ */
+ public static MemoryStateBackend defaultInstance() {
+ return DEFAULT_INSTANCE;
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/SerializedStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/SerializedStateHandle.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/SerializedStateHandle.java
new file mode 100644
index 0000000..163cadd
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/SerializedStateHandle.java
@@ -0,0 +1,49 @@
+/*
+ * 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.state.memory;
+
+import org.apache.flink.runtime.state.StateHandle;
+import org.apache.flink.util.SerializedValue;
+
+import java.io.IOException;
+
+/**
+ * A state handle that represents its state in serialized form as bytes.
+ *
+ * @param <T> The type of state represented by this state handle.
+ */
+public class SerializedStateHandle<T> extends SerializedValue<T> implements StateHandle<T> {
+
+ private static final long serialVersionUID = 4145685722538475769L;
+
+ public SerializedStateHandle(T value) throws IOException {
+ super(value);
+ }
+
+ @Override
+ public T getState(ClassLoader classLoader) throws Exception {
+ return deserializeValue(classLoader);
+ }
+
+ /**
+ * Discarding heap-memory backed state is a no-op, so this method does nothing.
+ */
+ @Override
+ public void discardState() {}
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/CoFeedbackTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/CoFeedbackTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/CoFeedbackTransformation.java
index 67ccbd6..f9c95f5 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/CoFeedbackTransformation.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/CoFeedbackTransformation.java
@@ -19,7 +19,7 @@ package org.apache.flink.streaming.api.transformations;
import com.google.common.collect.Lists;
import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
import java.util.Collection;
import java.util.Collections;
@@ -110,7 +110,7 @@ public class CoFeedbackTransformation<F> extends StreamTransformation<F> {
}
@Override
- public final void setChainingStrategy(StreamOperator.ChainingStrategy strategy) {
+ public final void setChainingStrategy(ChainingStrategy strategy) {
throw new UnsupportedOperationException("Cannot set chaining strategy on Split Transformation.");
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/FeedbackTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/FeedbackTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/FeedbackTransformation.java
index 11a2f33..87c7f16 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/FeedbackTransformation.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/FeedbackTransformation.java
@@ -18,7 +18,7 @@
package org.apache.flink.streaming.api.transformations;
import com.google.common.collect.Lists;
-import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
import java.util.Collection;
import java.util.List;
@@ -109,7 +109,7 @@ public class FeedbackTransformation<T> extends StreamTransformation<T> {
}
@Override
- public final void setChainingStrategy(StreamOperator.ChainingStrategy strategy) {
+ public final void setChainingStrategy(ChainingStrategy strategy) {
throw new UnsupportedOperationException("Cannot set chaining strategy on Split Transformation.");
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/OneInputTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/OneInputTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/OneInputTransformation.java
index 945d8eb..031c481 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/OneInputTransformation.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/OneInputTransformation.java
@@ -20,8 +20,8 @@ package org.apache.flink.streaming.api.transformations;
import com.google.common.collect.Lists;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.operators.StreamOperator;
import java.util.Collection;
import java.util.List;
@@ -38,10 +38,12 @@ public class OneInputTransformation<IN, OUT> extends StreamTransformation<OUT> {
private final StreamTransformation<IN> input;
- private KeySelector<IN, ?> stateKeySelector;
-
private final OneInputStreamOperator<IN, OUT> operator;
+ private KeySelector<IN, ?> stateKeySelector;
+
+ private TypeInformation<?> stateKeyType;
+
/**
* Creates a new {@code OneInputTransformation} from the given input and operator.
*
@@ -102,6 +104,14 @@ public class OneInputTransformation<IN, OUT> extends StreamTransformation<OUT> {
return stateKeySelector;
}
+ public void setStateKeyType(TypeInformation<?> stateKeyType) {
+ this.stateKeyType = stateKeyType;
+ }
+
+ public TypeInformation<?> getStateKeyType() {
+ return stateKeyType;
+ }
+
@Override
public Collection<StreamTransformation<?>> getTransitivePredecessors() {
List<StreamTransformation<?>> result = Lists.newArrayList();
@@ -111,7 +121,7 @@ public class OneInputTransformation<IN, OUT> extends StreamTransformation<OUT> {
}
@Override
- public final void setChainingStrategy(StreamOperator.ChainingStrategy strategy) {
+ public final void setChainingStrategy(ChainingStrategy strategy) {
operator.setChainingStrategy(strategy);
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/PartitionTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/PartitionTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/PartitionTransformation.java
index 1165d5d..fa85349 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/PartitionTransformation.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/PartitionTransformation.java
@@ -18,7 +18,8 @@
package org.apache.flink.streaming.api.transformations;
import com.google.common.collect.Lists;
-import org.apache.flink.streaming.api.operators.StreamOperator;
+
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
import java.util.Collection;
@@ -34,6 +35,7 @@ import java.util.List;
* @param <T> The type of the elements that result from this {@code PartitionTransformation}
*/
public class PartitionTransformation<T> extends StreamTransformation<T> {
+
private final StreamTransformation<T> input;
private final StreamPartitioner<T> partitioner;
@@ -74,7 +76,7 @@ public class PartitionTransformation<T> extends StreamTransformation<T> {
}
@Override
- public final void setChainingStrategy(StreamOperator.ChainingStrategy strategy) {
+ public final void setChainingStrategy(ChainingStrategy strategy) {
throw new UnsupportedOperationException("Cannot set chaining strategy on Union Transformation.");
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SelectTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SelectTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SelectTransformation.java
index 92033bd..a66b65a 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SelectTransformation.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SelectTransformation.java
@@ -18,7 +18,7 @@
package org.apache.flink.streaming.api.transformations;
import com.google.common.collect.Lists;
-import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
import java.util.Collection;
import java.util.List;
@@ -35,11 +35,12 @@ import java.util.List;
* @param <T> The type of the elements that result from this {@code SelectTransformation}
*/
public class SelectTransformation<T> extends StreamTransformation<T> {
+
private final StreamTransformation<T> input;
- private List<String> selectedNames;
+ private final List<String> selectedNames;
/**
- * Creates a new {@coe SelectionTransformation} from the given input that only selects
+ * Creates a new {@code SelectionTransformation} from the given input that only selects
* the streams with the selected names.
*
* @param input The input {@code StreamTransformation}
@@ -76,7 +77,7 @@ public class SelectTransformation<T> extends StreamTransformation<T> {
}
@Override
- public final void setChainingStrategy(StreamOperator.ChainingStrategy strategy) {
+ public final void setChainingStrategy(ChainingStrategy strategy) {
throw new UnsupportedOperationException("Cannot set chaining strategy on Select Transformation.");
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SinkTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SinkTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SinkTransformation.java
index 2a4e2d0..84ad6db 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SinkTransformation.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SinkTransformation.java
@@ -18,9 +18,10 @@
package org.apache.flink.streaming.api.transformations;
import com.google.common.collect.Lists;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
import org.apache.flink.streaming.api.operators.StreamSink;
import java.util.Collection;
@@ -39,6 +40,8 @@ public class SinkTransformation<T> extends StreamTransformation<Object> {
// We need this because sinks can also have state that is partitioned by key
private KeySelector<T, ?> stateKeySelector;
+
+ private TypeInformation<?> stateKeyType;
/**
* Creates a new {@code SinkTransformation} from the given input {@code StreamTransformation}.
@@ -91,6 +94,14 @@ public class SinkTransformation<T> extends StreamTransformation<Object> {
return stateKeySelector;
}
+ public void setStateKeyType(TypeInformation<?> stateKeyType) {
+ this.stateKeyType = stateKeyType;
+ }
+
+ public TypeInformation<?> getStateKeyType() {
+ return stateKeyType;
+ }
+
@Override
public Collection<StreamTransformation<?>> getTransitivePredecessors() {
List<StreamTransformation<?>> result = Lists.newArrayList();
@@ -100,7 +111,7 @@ public class SinkTransformation<T> extends StreamTransformation<Object> {
}
@Override
- public final void setChainingStrategy(StreamOperator.ChainingStrategy strategy) {
+ public final void setChainingStrategy(ChainingStrategy strategy) {
operator.setChainingStrategy(strategy);
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SourceTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SourceTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SourceTransformation.java
index c14c58c..9835606 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SourceTransformation.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SourceTransformation.java
@@ -18,7 +18,7 @@
package org.apache.flink.streaming.api.transformations;
import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
import org.apache.flink.streaming.api.operators.StreamSource;
import java.util.Collection;
@@ -64,7 +64,7 @@ public class SourceTransformation<T> extends StreamTransformation<T> {
}
@Override
- public final void setChainingStrategy(StreamOperator.ChainingStrategy strategy) {
+ public final void setChainingStrategy(ChainingStrategy strategy) {
operator.setChainingStrategy(strategy);
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SplitTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SplitTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SplitTransformation.java
index d392fd5..96c1c9e 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SplitTransformation.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/SplitTransformation.java
@@ -19,7 +19,7 @@ package org.apache.flink.streaming.api.transformations;
import com.google.common.collect.Lists;
import org.apache.flink.streaming.api.collector.selector.OutputSelector;
-import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
import java.util.Collection;
import java.util.List;
@@ -77,7 +77,7 @@ public class SplitTransformation<T> extends StreamTransformation<T> {
}
@Override
- public final void setChainingStrategy(StreamOperator.ChainingStrategy strategy) {
+ public final void setChainingStrategy(ChainingStrategy strategy) {
throw new UnsupportedOperationException("Cannot set chaining strategy on Split Transformation.");
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java
index dadcfa2..4e6dc42 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/StreamTransformation.java
@@ -18,11 +18,12 @@
package org.apache.flink.streaming.api.transformations;
import com.google.common.base.Preconditions;
+
import org.apache.flink.api.common.functions.InvalidTypesException;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.typeutils.MissingTypeInfo;
import org.apache.flink.streaming.api.graph.StreamGraph;
-import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
import java.util.Collection;
@@ -216,7 +217,7 @@ public abstract class StreamTransformation<T> {
/**
* Sets the chaining strategy of this {@code StreamTransformation}.
*/
- public abstract void setChainingStrategy(StreamOperator.ChainingStrategy strategy);
+ public abstract void setChainingStrategy(ChainingStrategy strategy);
/**
* Set the buffer timeout of this {@code StreamTransformation}. The timeout is used when
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/TwoInputTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/TwoInputTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/TwoInputTransformation.java
index e7273c5..30f0733 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/TwoInputTransformation.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/TwoInputTransformation.java
@@ -19,7 +19,7 @@ package org.apache.flink.streaming.api.transformations;
import com.google.common.collect.Lists;
import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
import java.util.Collection;
@@ -109,7 +109,7 @@ public class TwoInputTransformation<IN1, IN2, OUT> extends StreamTransformation<
}
@Override
- public final void setChainingStrategy(StreamOperator.ChainingStrategy strategy) {
+ public final void setChainingStrategy(ChainingStrategy strategy) {
operator.setChainingStrategy(strategy);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/UnionTransformation.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/UnionTransformation.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/UnionTransformation.java
index 4fa3c0a..3e1ff57 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/UnionTransformation.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/transformations/UnionTransformation.java
@@ -18,7 +18,7 @@
package org.apache.flink.streaming.api.transformations;
import com.google.common.collect.Lists;
-import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
import java.util.Collection;
import java.util.List;
@@ -74,7 +74,7 @@ public class UnionTransformation<T> extends StreamTransformation<T> {
}
@Override
- public final void setChainingStrategy(StreamOperator.ChainingStrategy strategy) {
+ public final void setChainingStrategy(ChainingStrategy strategy) {
throw new UnsupportedOperationException("Cannot set chaining strategy on Union Transformation.");
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CollectorWrapper.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CollectorWrapper.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CollectorWrapper.java
index 6bb44dd..01e997d 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CollectorWrapper.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/CollectorWrapper.java
@@ -25,23 +25,21 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.util.Collector;
import java.util.ArrayList;
-import java.util.List;
public class CollectorWrapper<OUT> implements Output<StreamRecord<OUT>> {
private OutputSelectorWrapper<OUT> outputSelectorWrapper;
- private List<Output<OUT>> allOutputs;
+ private ArrayList<Output<StreamRecord<OUT>>> allOutputs;
public CollectorWrapper(OutputSelectorWrapper<OUT> outputSelectorWrapper) {
this.outputSelectorWrapper = outputSelectorWrapper;
- allOutputs = new ArrayList<Output<OUT>>();
+ allOutputs = new ArrayList<Output<StreamRecord<OUT>>>();
}
-
- @SuppressWarnings("unchecked,rawtypes")
- public void addCollector(Output<StreamRecord<?>> output, StreamEdge edge) {
+
+ public void addCollector(Output<StreamRecord<OUT>> output, StreamEdge edge) {
outputSelectorWrapper.addCollector(output, edge);
- allOutputs.add((Output) output);
+ allOutputs.add(output);
}
@Override
@@ -53,13 +51,11 @@ public class CollectorWrapper<OUT> implements Output<StreamRecord<OUT>> {
@Override
public void emitWatermark(Watermark mark) {
- for (Output<OUT> output : allOutputs) {
+ for (Output<?> output : allOutputs) {
output.emitWatermark(mark);
}
}
@Override
- public void close() {
- }
-
+ public void close() {}
}
[23/24] flink git commit: [FLINK-2550] [tests] Add an end-to-end
failure/recovery test for fast path processing time windows
Posted by se...@apache.org.
[FLINK-2550] [tests] Add an end-to-end failure/recovery test for fast path processing time windows
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/3b7fb631
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/3b7fb631
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/3b7fb631
Branch: refs/heads/master
Commit: 3b7fb631aec2f2ffd599b7b0cd5c0abd5d1f10dd
Parents: da159ef
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Oct 9 22:16:59 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 16 15:26:11 2015 +0200
----------------------------------------------------------------------
.../WindowCheckpointingITCase.java | 501 +++++++++++++++++++
1 file changed, 501 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/3b7fb631/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java
new file mode 100644
index 0000000..298ae5c
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/WindowCheckpointingITCase.java
@@ -0,0 +1,501 @@
+/*
+ * 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.test.checkpointing;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.functions.RichReduceFunction;
+import org.apache.flink.api.java.tuple.Tuple;
+import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.client.program.ProgramInvocationException;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.StreamingMode;
+import org.apache.flink.runtime.client.JobExecutionException;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier;
+import org.apache.flink.streaming.api.checkpoint.Checkpointed;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction;
+import org.apache.flink.streaming.api.windowing.time.Time;
+import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
+import org.apache.flink.test.util.ForkableFlinkMiniCluster;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.TestLogger;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.HashMap;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * This test uses a custom non-serializable data type to to ensure that state
+ * serializability is handled correctly.
+ */
+@SuppressWarnings("serial")
+public class WindowCheckpointingITCase extends TestLogger {
+
+ private static final int PARALLELISM = 4;
+
+ private static ForkableFlinkMiniCluster cluster;
+
+
+ @BeforeClass
+ public static void startTestCluster() {
+ Configuration config = new Configuration();
+ config.setInteger(ConfigConstants.LOCAL_NUMBER_TASK_MANAGER, 2);
+ config.setInteger(ConfigConstants.TASK_MANAGER_NUM_TASK_SLOTS, PARALLELISM / 2);
+ config.setInteger(ConfigConstants.TASK_MANAGER_MEMORY_SIZE_KEY, 48);
+ config.setString(ConfigConstants.DEFAULT_EXECUTION_RETRY_DELAY_KEY, "0 ms");
+
+ cluster = new ForkableFlinkMiniCluster(config, false, StreamingMode.STREAMING);
+ cluster.start();
+ }
+
+ @AfterClass
+ public static void stopTestCluster() {
+ if (cluster != null) {
+ cluster.stop();
+ }
+ }
+
+ // ------------------------------------------------------------------------
+
+ @Test
+ public void testTumblingProcessingTimeWindow() {
+ final int NUM_ELEMENTS = 3000;
+ FailingSource.reset();
+
+ try {
+ StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
+ "localhost", cluster.getLeaderRPCPort());
+
+ env.setParallelism(PARALLELISM);
+ env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime);
+ env.enableCheckpointing(100);
+ env.setNumberOfExecutionRetries(3);
+ env.getConfig().disableSysoutLogging();
+
+ env
+ .addSource(new FailingSource(NUM_ELEMENTS, NUM_ELEMENTS / 3))
+ .rebalance()
+ .keyBy(0)
+ .timeWindow(Time.of(100, MILLISECONDS))
+ .apply(new RichWindowFunction<Tuple2<Long, IntType>, Tuple2<Long, IntType>, Tuple, TimeWindow>() {
+
+ private boolean open = false;
+
+ @Override
+ public void open(Configuration parameters) {
+ assertEquals(PARALLELISM, getRuntimeContext().getNumberOfParallelSubtasks());
+ open = true;
+ }
+
+ @Override
+ public void apply(
+ Tuple tuple,
+ TimeWindow window,
+ Iterable<Tuple2<Long, IntType>> values,
+ Collector<Tuple2<Long, IntType>> out) {
+
+ // validate that the function has been opened properly
+ assertTrue(open);
+
+ for (Tuple2<Long, IntType> value : values) {
+ assertEquals(value.f0.intValue(), value.f1.value);
+ out.collect(new Tuple2<Long, IntType>(value.f0, new IntType(1)));
+ }
+ }
+ })
+ .addSink(new ValidatingSink(NUM_ELEMENTS, 1)).setParallelism(1);
+
+
+ tryExecute(env, "Tumbling Window Test");
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testSlidingProcessingTimeWindow() {
+ final int NUM_ELEMENTS = 3000;
+ FailingSource.reset();
+
+ try {
+ StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
+ "localhost", cluster.getLeaderRPCPort());
+
+ env.setParallelism(PARALLELISM);
+ env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime);
+ env.enableCheckpointing(100);
+ env.setNumberOfExecutionRetries(3);
+ env.getConfig().disableSysoutLogging();
+
+ env
+ .addSource(new FailingSource(NUM_ELEMENTS, NUM_ELEMENTS / 3))
+ .rebalance()
+ .keyBy(0)
+ .timeWindow(Time.of(150, MILLISECONDS), Time.of(50, MILLISECONDS))
+ .apply(new RichWindowFunction<Tuple2<Long, IntType>, Tuple2<Long, IntType>, Tuple, TimeWindow>() {
+
+ private boolean open = false;
+
+ @Override
+ public void open(Configuration parameters) {
+ assertEquals(PARALLELISM, getRuntimeContext().getNumberOfParallelSubtasks());
+ open = true;
+ }
+
+ @Override
+ public void apply(
+ Tuple tuple,
+ TimeWindow window,
+ Iterable<Tuple2<Long, IntType>> values,
+ Collector<Tuple2<Long, IntType>> out) {
+
+ // validate that the function has been opened properly
+ assertTrue(open);
+
+ for (Tuple2<Long, IntType> value : values) {
+ assertEquals(value.f0.intValue(), value.f1.value);
+ out.collect(new Tuple2<Long, IntType>(value.f0, new IntType(1)));
+ }
+ }
+ })
+ .addSink(new ValidatingSink(NUM_ELEMENTS, 3)).setParallelism(1);
+
+
+ tryExecute(env, "Tumbling Window Test");
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testAggregatingTumblingProcessingTimeWindow() {
+ final int NUM_ELEMENTS = 3000;
+ FailingSource.reset();
+
+ try {
+ StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
+ "localhost", cluster.getLeaderRPCPort());
+
+ env.setParallelism(PARALLELISM);
+ env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime);
+ env.enableCheckpointing(100);
+ env.setNumberOfExecutionRetries(3);
+ env.getConfig().disableSysoutLogging();
+
+ env
+ .addSource(new FailingSource(NUM_ELEMENTS, NUM_ELEMENTS / 3))
+ .map(new MapFunction<Tuple2<Long,IntType>, Tuple2<Long,IntType>>() {
+ @Override
+ public Tuple2<Long, IntType> map(Tuple2<Long, IntType> value) {
+ value.f1.value = 1;
+ return value;
+ }
+ })
+ .rebalance()
+ .keyBy(0)
+ .timeWindow(Time.of(100, MILLISECONDS))
+ .reduce(new RichReduceFunction<Tuple2<Long, IntType>>() {
+
+ private boolean open = false;
+
+ @Override
+ public void open(Configuration parameters) {
+ assertEquals(PARALLELISM, getRuntimeContext().getNumberOfParallelSubtasks());
+ open = true;
+ }
+
+ @Override
+ public Tuple2<Long, IntType> reduce(
+ Tuple2<Long, IntType> a,
+ Tuple2<Long, IntType> b) {
+
+ // validate that the function has been opened properly
+ assertTrue(open);
+ return new Tuple2<>(a.f0, new IntType(1));
+ }
+ })
+ .addSink(new ValidatingSink(NUM_ELEMENTS, 1)).setParallelism(1);
+
+
+ tryExecute(env, "Tumbling Window Test");
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testAggregatingSlidingProcessingTimeWindow() {
+ final int NUM_ELEMENTS = 3000;
+ FailingSource.reset();
+
+ try {
+ StreamExecutionEnvironment env = StreamExecutionEnvironment.createRemoteEnvironment(
+ "localhost", cluster.getLeaderRPCPort());
+
+ env.setParallelism(PARALLELISM);
+ env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime);
+ env.enableCheckpointing(100);
+ env.setNumberOfExecutionRetries(3);
+ env.getConfig().disableSysoutLogging();
+
+ env
+ .addSource(new FailingSource(NUM_ELEMENTS, NUM_ELEMENTS / 3))
+ .map(new MapFunction<Tuple2<Long,IntType>, Tuple2<Long,IntType>>() {
+ @Override
+ public Tuple2<Long, IntType> map(Tuple2<Long, IntType> value) {
+ value.f1.value = 1;
+ return value;
+ }
+ })
+ .rebalance()
+ .keyBy(0)
+ .timeWindow(Time.of(150, MILLISECONDS), Time.of(50, MILLISECONDS))
+ .reduce(new RichReduceFunction<Tuple2<Long, IntType>>() {
+
+ private boolean open = false;
+
+ @Override
+ public void open(Configuration parameters) {
+ assertEquals(PARALLELISM, getRuntimeContext().getNumberOfParallelSubtasks());
+ open = true;
+ }
+
+ @Override
+ public Tuple2<Long, IntType> reduce(
+ Tuple2<Long, IntType> a,
+ Tuple2<Long, IntType> b) {
+
+ // validate that the function has been opened properly
+ assertTrue(open);
+ return new Tuple2<>(a.f0, new IntType(1));
+ }
+ })
+ .addSink(new ValidatingSink(NUM_ELEMENTS, 3)).setParallelism(1);
+
+
+ tryExecute(env, "Tumbling Window Test");
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ // ------------------------------------------------------------------------
+ // Utilities
+ // ------------------------------------------------------------------------
+
+ private static class FailingSource extends RichSourceFunction<Tuple2<Long, IntType>>
+ implements Checkpointed<Integer>, CheckpointNotifier
+ {
+ private static volatile boolean failedBefore = false;
+
+ private final int numElementsToEmit;
+ private final int failureAfterNumElements;
+
+ private volatile int numElementsEmitted;
+ private volatile int numSuccessfulCheckpoints;
+ private volatile boolean running = true;
+
+ private FailingSource(int numElementsToEmit, int failureAfterNumElements) {
+ this.numElementsToEmit = numElementsToEmit;
+ this.failureAfterNumElements = failureAfterNumElements;
+ }
+
+ @Override
+ public void open(Configuration parameters) {
+ // non-parallel source
+ assertEquals(1, getRuntimeContext().getNumberOfParallelSubtasks());
+ }
+
+ @Override
+ public void run(SourceContext<Tuple2<Long, IntType>> ctx) throws Exception {
+ // we loop longer than we have elements, to permit delayed checkpoints
+ // to still cause a failure
+ while (running) {
+
+ if (!failedBefore) {
+ // delay a bit, if we have not failed before
+ Thread.sleep(1);
+ if (numSuccessfulCheckpoints >= 2 && numElementsEmitted >= failureAfterNumElements) {
+ // cause a failure if we have not failed before and have reached
+ // enough completed checkpoints and elements
+ failedBefore = true;
+ throw new Exception("Artificial Failure");
+ }
+ }
+
+ if (numElementsEmitted < numElementsToEmit &&
+ (failedBefore || numElementsEmitted <= failureAfterNumElements))
+ {
+ // the function failed before, or we are in the elements before the failure
+ synchronized (ctx.getCheckpointLock()) {
+ int next = numElementsEmitted++;
+ ctx.collect(new Tuple2<Long, IntType>((long) next, new IntType(next)));
+ }
+ }
+ else {
+ // if our work is done, delay a bit to prevent busy waiting
+ Thread.sleep(1);
+ }
+ }
+ }
+
+ @Override
+ public void cancel() {
+ running = false;
+ }
+
+ @Override
+ public void notifyCheckpointComplete(long checkpointId) {
+ numSuccessfulCheckpoints++;
+ }
+
+ @Override
+ public Integer snapshotState(long checkpointId, long checkpointTimestamp) {
+ return numElementsEmitted;
+ }
+
+ @Override
+ public void restoreState(Integer state) {
+ numElementsEmitted = state;
+ }
+
+ public static void reset() {
+ failedBefore = false;
+ }
+ }
+
+ private static class ValidatingSink extends RichSinkFunction<Tuple2<Long, IntType>>
+ implements Checkpointed<HashMap<Long, Integer>> {
+
+ private final HashMap<Long, Integer> counts = new HashMap<>();
+
+ private final int elementCountExpected;
+ private final int countPerElementExpected;
+
+ private int aggCount;
+
+ private ValidatingSink(int elementCountExpected, int countPerElementExpected) {
+ this.elementCountExpected = elementCountExpected;
+ this.countPerElementExpected = countPerElementExpected;
+ }
+
+ @Override
+ public void open(Configuration parameters) throws Exception {
+ // this sink can only work with DOP 1
+ assertEquals(1, getRuntimeContext().getNumberOfParallelSubtasks());
+ }
+
+ @Override
+ public void invoke(Tuple2<Long, IntType> value) throws Exception {
+ Integer curr = counts.get(value.f0);
+ if (curr != null) {
+ counts.put(value.f0, curr + value.f1.value);
+ }
+ else {
+ counts.put(value.f0, value.f1.value);
+ }
+
+ // check if we have seen all we expect
+ aggCount += value.f1.value;
+ if (aggCount >= elementCountExpected * countPerElementExpected) {
+ // we are done. validate
+ assertEquals(elementCountExpected, counts.size());
+
+ for (Integer i : counts.values()) {
+ assertEquals(countPerElementExpected, i.intValue());
+ }
+
+ // exit
+ throw new SuccessException();
+ }
+ }
+
+ @Override
+ public HashMap<Long, Integer> snapshotState(long checkpointId, long checkpointTimestamp) {
+ return this.counts;
+ }
+
+ @Override
+ public void restoreState(HashMap<Long, Integer> state) {
+ this.counts.putAll(state);
+
+ for (Integer i : state.values()) {
+ this.aggCount += i;
+ }
+
+ }
+ }
+
+ // ------------------------------------------------------------------------
+ // Utilities
+ // ------------------------------------------------------------------------
+
+ public static void tryExecute(StreamExecutionEnvironment env, String jobName) throws Exception {
+ try {
+ env.execute(jobName);
+ }
+ catch (ProgramInvocationException | JobExecutionException root) {
+ Throwable cause = root.getCause();
+
+ // search for nested SuccessExceptions
+ int depth = 0;
+ while (!(cause instanceof SuccessException)) {
+ if (cause == null || depth++ == 20) {
+ root.printStackTrace();
+ fail("Test failed: " + root.getMessage());
+ }
+ else {
+ cause = cause.getCause();
+ }
+ }
+ }
+ }
+
+ public static class IntType {
+
+ public int value;
+
+ public IntType() {}
+
+ public IntType(int value) { this.value = value; }
+ }
+
+ static final class SuccessException extends Exception {
+ private static final long serialVersionUID = -9218191172606739598L;
+ }
+}
\ No newline at end of file
[19/24] flink git commit: [FLINK-2550] [streaming] Allow multiple
key/value states per operator on top of the new state backend
Posted by se...@apache.org.
[FLINK-2550] [streaming] Allow multiple key/value states per operator on top of the new state backend
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/7c205432
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/7c205432
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/7c205432
Branch: refs/heads/master
Commit: 7c2054326b259bde0e8dd28d3428e63fd285454c
Parents: bb1f5fd
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Oct 9 23:59:28 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 16 15:26:11 2015 +0200
----------------------------------------------------------------------
.../flink/storm/api/FlinkTopologyBuilder.java | 1 +
.../storm/util/SplitStreamTypeKeySelector.java | 5 +-
.../storm/api/FlinkTopologyBuilderTest.java | 3 -
.../api/common/functions/RuntimeContext.java | 10 +-
.../util/AbstractRuntimeUDFContext.java | 4 +-
.../common/typeinfo/PrimitiveArrayTypeInfo.java | 16 +++
.../kafka/testutils/MockRuntimeContext.java | 4 +-
.../streaming/api/datastream/DataStream.java | 4 +-
.../api/operators/AbstractStreamOperator.java | 102 ++++++++++++-------
.../api/operators/StreamGroupedFold.java | 4 +-
.../api/operators/StreamGroupedReduce.java | 4 +-
.../api/operators/StreamingRuntimeContext.java | 38 ++++---
.../runtime/tasks/StreamTaskState.java | 36 +++++--
.../streaming/util/keys/KeySelectorUtil.java | 94 +++++++++++++----
.../api/ChainedRuntimeContextTest.java | 3 +-
.../util/keys/ArrayKeySelectorTest.java | 20 ++--
.../streaming/api/scala/ConnectedStreams.scala | 1 -
.../flink/streaming/api/scala/KeyedStream.scala | 9 ++
.../api/scala/function/StatefulFunction.scala | 2 +-
.../streaming/api/scala/DataStreamTest.scala | 2 +-
.../streaming/api/scala/StateTestPrograms.scala | 24 +++--
.../PartitionedStateCheckpointingITCase.java | 23 +++--
.../StreamCheckpointingITCase.java | 2 +-
23 files changed, 292 insertions(+), 119 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/7c205432/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkTopologyBuilder.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkTopologyBuilder.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkTopologyBuilder.java
index 99de0e2..9c41d88 100644
--- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkTopologyBuilder.java
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/api/FlinkTopologyBuilder.java
@@ -200,6 +200,7 @@ public class FlinkTopologyBuilder {
} else {
inputStream = inputStream
.keyBy(new SplitStreamTypeKeySelector(
+ inputStream.getType(),
prodDeclarer.getGroupingFieldIndexes(
inputStreamId,
grouping.get_fields())));
http://git-wip-us.apache.org/repos/asf/flink/blob/7c205432/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SplitStreamTypeKeySelector.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SplitStreamTypeKeySelector.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SplitStreamTypeKeySelector.java
index 44c693c..71e5b86 100644
--- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SplitStreamTypeKeySelector.java
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/util/SplitStreamTypeKeySelector.java
@@ -17,6 +17,7 @@
*/
package org.apache.flink.storm.util;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.streaming.util.keys.KeySelectorUtil;
@@ -34,8 +35,8 @@ public class SplitStreamTypeKeySelector implements KeySelector<SplitStreamType<T
private final ArrayKeySelector<Tuple> selector;
- public SplitStreamTypeKeySelector(int... fields) {
- this.selector = new KeySelectorUtil.ArrayKeySelector<Tuple>(fields);
+ public SplitStreamTypeKeySelector(TypeInformation<Tuple> type, int... fields) {
+ this.selector = KeySelectorUtil.getSelectorForArray(fields, type);
}
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/7c205432/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyBuilderTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyBuilderTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyBuilderTest.java
index 906d081..fa5c8d8 100644
--- a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyBuilderTest.java
+++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyBuilderTest.java
@@ -21,7 +21,6 @@ import org.apache.flink.storm.util.TestDummyBolt;
import org.apache.flink.storm.util.TestDummySpout;
import org.apache.flink.storm.util.TestSink;
-import org.junit.Ignore;
import org.junit.Test;
import backtype.storm.tuple.Fields;
@@ -54,7 +53,6 @@ public class FlinkTopologyBuilderTest {
}
@Test
- @Ignore
public void testFieldsGroupingOnMultipleSpoutOutputStreams() {
FlinkTopologyBuilder flinkBuilder = new FlinkTopologyBuilder();
@@ -66,7 +64,6 @@ public class FlinkTopologyBuilderTest {
}
@Test
- @Ignore
public void testFieldsGroupingOnMultipleBoltOutputStreams() {
FlinkTopologyBuilder flinkBuilder = new FlinkTopologyBuilder();
http://git-wip-us.apache.org/repos/asf/flink/blob/7c205432/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java
index cadef36..7f767c3 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java
@@ -167,6 +167,7 @@ public interface RuntimeContext {
* Gets the key/value state, which is only accessible if the function is executed on
* a KeyedStream. Upon calling {@link OperatorState#value()}, the key/value state will
* return the value bound to the key of the element currently processed by the function.
+ * Each operator may maintain multiple key/value states, addressed with different names.
*
* <p>Because the scope of each value is the key of the currently processed element,
* and the elements are distributed by the Flink runtime, the system can transparently
@@ -200,8 +201,9 @@ public interface RuntimeContext {
* <p>This method attempts to deduce the type information from the given type class. If the
* full type cannot be determined from the class (for example because of generic parameters),
* the TypeInformation object must be manually passed via
- * {@link #getKeyValueState(TypeInformation, Object)}.
+ * {@link #getKeyValueState(String, TypeInformation, Object)}.
*
+ * @param name The name of the key/value state.
* @param stateType The class of the type that is stored in the state. Used to generate
* serializers for managed memory and checkpointing.
* @param defaultState The default state value, returned when the state is accessed and
@@ -213,12 +215,13 @@ public interface RuntimeContext {
* @throws UnsupportedOperationException Thrown, if no key/value state is available for the
* function (function is not part os a KeyedStream).
*/
- <S> OperatorState<S> getKeyValueState(Class<S> stateType, S defaultState);
+ <S> OperatorState<S> getKeyValueState(String name, Class<S> stateType, S defaultState);
/**
* Gets the key/value state, which is only accessible if the function is executed on
* a KeyedStream. Upon calling {@link OperatorState#value()}, the key/value state will
* return the value bound to the key of the element currently processed by the function.
+ * Each operator may maintain multiple key/value states, addressed with different names.
*
* <p>Because the scope of each value is the key of the currently processed element,
* and the elements are distributed by the Flink runtime, the system can transparently
@@ -249,6 +252,7 @@ public interface RuntimeContext {
*
* }</pre>
*
+ * @param name The name of the key/value state.
* @param stateType The type information for the type that is stored in the state.
* Used to create serializers for managed memory and checkpoints.
* @param defaultState The default state value, returned when the state is accessed and
@@ -260,5 +264,5 @@ public interface RuntimeContext {
* @throws UnsupportedOperationException Thrown, if no key/value state is available for the
* function (function is not part os a KeyedStream).
*/
- <S> OperatorState<S> getKeyValueState(TypeInformation<S> stateType, S defaultState);
+ <S> OperatorState<S> getKeyValueState(String name, TypeInformation<S> stateType, S defaultState);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/7c205432/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java
index 90d23cd..be8ac9d 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java
@@ -165,13 +165,13 @@ public abstract class AbstractRuntimeUDFContext implements RuntimeContext {
}
@Override
- public <S> OperatorState<S> getKeyValueState(Class<S> stateType, S defaultState) {
+ public <S> OperatorState<S> getKeyValueState(String name, Class<S> stateType, S defaultState) {
throw new UnsupportedOperationException(
"This state is only accessible by functions executed on a KeyedStream");
}
@Override
- public <S> OperatorState<S> getKeyValueState(TypeInformation<S> stateType, S defaultState) {
+ public <S> OperatorState<S> getKeyValueState(String name, TypeInformation<S> stateType, S defaultState) {
throw new UnsupportedOperationException(
"This state is only accessible by functions executed on a KeyedStream");
}
http://git-wip-us.apache.org/repos/asf/flink/blob/7c205432/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/PrimitiveArrayTypeInfo.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/PrimitiveArrayTypeInfo.java b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/PrimitiveArrayTypeInfo.java
index 44339ac..9bb444a 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/PrimitiveArrayTypeInfo.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeinfo/PrimitiveArrayTypeInfo.java
@@ -125,6 +125,22 @@ public class PrimitiveArrayTypeInfo<T> extends TypeInformation<T> implements Ato
public TypeSerializer<T> createSerializer(ExecutionConfig executionConfig) {
return this.serializer;
}
+
+ /**
+ * Gets the class that represents the component type.
+ * @return The class of the component type.
+ */
+ public Class<?> getComponentClass() {
+ return this.arrayClass.getComponentType();
+ }
+
+ /**
+ * Gets the type information of the component type.
+ * @return The type information of the component type.
+ */
+ public TypeInformation<?> getComponentType() {
+ return BasicTypeInfo.getInfoFor(getComponentClass());
+ }
@Override
public String toString() {
http://git-wip-us.apache.org/repos/asf/flink/blob/7c205432/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java
index 9718b72..b9fc3de 100644
--- a/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java
+++ b/flink-staging/flink-streaming/flink-streaming-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/testutils/MockRuntimeContext.java
@@ -121,12 +121,12 @@ public class MockRuntimeContext implements RuntimeContext {
}
@Override
- public <S> OperatorState<S> getKeyValueState(Class<S> stateType, S defaultState) {
+ public <S> OperatorState<S> getKeyValueState(String name, Class<S> stateType, S defaultState) {
throw new UnsupportedOperationException();
}
@Override
- public <S> OperatorState<S> getKeyValueState(TypeInformation<S> stateType, S defaultState) {
+ public <S> OperatorState<S> getKeyValueState(String name, TypeInformation<S> stateType, S defaultState) {
throw new UnsupportedOperationException();
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/7c205432/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
index c15ea9b..176a07f 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java
@@ -253,7 +253,7 @@ public class DataStream<T> {
*/
public KeyedStream<T, Tuple> keyBy(int... fields) {
if (getType() instanceof BasicArrayTypeInfo || getType() instanceof PrimitiveArrayTypeInfo) {
- return keyBy(new KeySelectorUtil.ArrayKeySelector<T>(fields));
+ return keyBy(KeySelectorUtil.getSelectorForArray(fields, getType()));
} else {
return keyBy(new Keys.ExpressionKeys<T>(fields, getType()));
}
@@ -291,7 +291,7 @@ public class DataStream<T> {
*/
public DataStream<T> partitionByHash(int... fields) {
if (getType() instanceof BasicArrayTypeInfo || getType() instanceof PrimitiveArrayTypeInfo) {
- return partitionByHash(new KeySelectorUtil.ArrayKeySelector<T>(fields));
+ return partitionByHash(KeySelectorUtil.getSelectorForArray(fields, getType()));
} else {
return partitionByHash(new Keys.ExpressionKeys<T>(fields, getType()));
}
http://git-wip-us.apache.org/repos/asf/flink/blob/7c205432/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
index ca86627..9e60e9a 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
@@ -35,6 +35,9 @@ import org.apache.flink.streaming.runtime.tasks.StreamTaskState;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import java.util.HashMap;
+import java.util.Map;
+
/**
* Base class for all stream operators. Operators that contain a user function should extend the class
* {@link AbstractUdfStreamOperator} instead (which is a specialized subclass of this class).
@@ -77,14 +80,19 @@ public abstract class AbstractStreamOperator<OUT>
/** The runtime context for UDFs */
private transient StreamingRuntimeContext runtimeContext;
+
// ---------------- key/value state ------------------
/** key selector used to get the key for the state. Non-null only is the operator uses key/value state */
private transient KeySelector<?, ?> stateKeySelector;
- private transient KvState<?, ?, ?> keyValueState;
+ private transient KvState<?, ?, ?>[] keyValueStates;
+
+ private transient HashMap<String, KvState<?, ?, ?>> keyValueStatesByName;
- private transient KvStateSnapshot<?, ?, ?> keyValueStateSnapshot;
+ private transient TypeSerializer<?> keySerializer;
+
+ private transient HashMap<String, KvStateSnapshot<?, ?, ?>> keyValueStateSnapshots;
// ------------------------------------------------------------------------
// Life Cycle
@@ -133,8 +141,10 @@ public abstract class AbstractStreamOperator<OUT>
*/
@Override
public void dispose() {
- if (keyValueState != null) {
- keyValueState.dispose();
+ if (keyValueStates != null) {
+ for (KvState<?, ?, ?> state : keyValueStates) {
+ state.dispose();
+ }
}
}
@@ -147,9 +157,15 @@ public abstract class AbstractStreamOperator<OUT>
// here, we deal with key/value state snapshots
StreamTaskState state = new StreamTaskState();
- if (keyValueState != null) {
- KvStateSnapshot<?, ?, ?> snapshot = keyValueState.shapshot(checkpointId, timestamp);
- state.setKvState(snapshot);
+ if (keyValueStates != null) {
+ HashMap<String, KvStateSnapshot<?, ?, ?>> snapshots = new HashMap<>(keyValueStatesByName.size());
+
+ for (Map.Entry<String, KvState<?, ?, ?>> entry : keyValueStatesByName.entrySet()) {
+ KvStateSnapshot<?, ?, ?> snapshot = entry.getValue().shapshot(checkpointId, timestamp);
+ snapshots.put(entry.getKey(), snapshot);
+ }
+
+ state.setKvStates(snapshots);
}
return state;
@@ -159,7 +175,7 @@ public abstract class AbstractStreamOperator<OUT>
public void restoreState(StreamTaskState state) throws Exception {
// restore the key/value state. the actual restore happens lazily, when the function requests
// the state again, because the restore method needs information provided by the user function
- keyValueStateSnapshot = state.getKvState();
+ keyValueStateSnapshots = state.getKvStates();
}
@Override
@@ -232,9 +248,9 @@ public abstract class AbstractStreamOperator<OUT>
* @throws Exception Thrown, if the state backend cannot create the key/value state.
*/
protected <V> OperatorState<V> createKeyValueState(
- TypeInformation<V> stateType, V defaultValue) throws Exception
+ String name, TypeInformation<V> stateType, V defaultValue) throws Exception
{
- return createKeyValueState(stateType.createSerializer(getExecutionConfig()), defaultValue);
+ return createKeyValueState(name, stateType.createSerializer(getExecutionConfig()), defaultValue);
}
/**
@@ -253,12 +269,18 @@ public abstract class AbstractStreamOperator<OUT>
* @throws IllegalStateException Thrown, if the key/value state was already initialized.
* @throws Exception Thrown, if the state backend cannot create the key/value state.
*/
+ @SuppressWarnings({"rawtypes", "unchecked"})
protected <K, V, Backend extends StateBackend<Backend>> OperatorState<V> createKeyValueState(
- TypeSerializer<V> valueSerializer, V defaultValue) throws Exception
+ String name, TypeSerializer<V> valueSerializer, V defaultValue) throws Exception
{
- if (keyValueState != null) {
+ if (name == null || name.isEmpty()) {
+ throw new IllegalArgumentException();
+ }
+ if (keyValueStatesByName != null && keyValueStatesByName.containsKey(name)) {
throw new IllegalStateException("The key/value state has already been created");
}
+
+ TypeSerializer<K> keySerializer;
// first time state access, make sure we load the state partitioner
if (stateKeySelector == null) {
@@ -267,46 +289,58 @@ public abstract class AbstractStreamOperator<OUT>
throw new UnsupportedOperationException("The function or operator is not executed " +
"on a KeyedStream and can hence not access the key/value state");
}
+
+ keySerializer = config.getStateKeySerializer(getUserCodeClassloader());
+ if (keySerializer == null) {
+ throw new Exception("State key serializer has not been configured in the config.");
+ }
+ this.keySerializer = keySerializer;
}
-
- // create the key and value serializers
- TypeSerializer<K> keySerializer = config.getStateKeySerializer(getUserCodeClassloader());
- if (keySerializer == null) {
- throw new Exception("State key serializer has not been configured in the config.");
+ else if (this.keySerializer != null) {
+ keySerializer = (TypeSerializer<K>) this.keySerializer;
+ }
+ else {
+ // should never happen, this is merely a safeguard
+ throw new RuntimeException();
}
@SuppressWarnings("unchecked")
Backend stateBackend = (Backend) container.getStateBackend();
+
+ KvState<K, V, Backend> kvstate = null;
// check whether we restore the key/value state from a snapshot, or create a new blank one
- if (keyValueStateSnapshot != null) {
+ if (keyValueStateSnapshots != null) {
@SuppressWarnings("unchecked")
- KvStateSnapshot<K, V, Backend> snapshot = (KvStateSnapshot<K, V, Backend>) keyValueStateSnapshot;
+ KvStateSnapshot<K, V, Backend> snapshot = (KvStateSnapshot<K, V, Backend>) keyValueStateSnapshots.remove(name);
- KvState<K, V, Backend> kvstate = snapshot.restoreState(
- stateBackend, keySerializer, valueSerializer, defaultValue, getUserCodeClassloader());
- keyValueState = kvstate;
-
- // make sure we have no redundant copies in memory, let the GC clean up
- keyValueStateSnapshot = null;
-
- return kvstate;
+ if (snapshot != null) {
+ kvstate = snapshot.restoreState(
+ stateBackend, keySerializer, valueSerializer, defaultValue, getUserCodeClassloader());
+ }
}
- else {
+
+ if (kvstate == null) {
// create a new blank key/value state
- KvState<K, V, Backend> kvstate = stateBackend.createKvState(keySerializer, valueSerializer, defaultValue);
- keyValueState = kvstate;
- return kvstate;
+ kvstate = stateBackend.createKvState(keySerializer, valueSerializer, defaultValue);
}
+
+ if (keyValueStatesByName == null) {
+ keyValueStatesByName = new HashMap<>();
+ }
+ keyValueStatesByName.put(name, kvstate);
+ keyValueStates = keyValueStatesByName.values().toArray(new KvState[keyValueStatesByName.size()]);
+ return kvstate;
}
@Override
@SuppressWarnings({"unchecked", "rawtypes"})
public void setKeyContextElement(StreamRecord record) throws Exception {
- if (stateKeySelector != null && keyValueState != null) {
- KvState kv = keyValueState;
+ if (stateKeySelector != null && keyValueStates != null) {
KeySelector selector = stateKeySelector;
- kv.setCurrentKey(selector.getKey(record.getValue()));
+ for (KvState kv : keyValueStates) {
+ kv.setCurrentKey(selector.getKey(record.getValue()));
+ }
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/7c205432/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedFold.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedFold.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedFold.java
index 79e319a..cf6b489 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedFold.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedFold.java
@@ -38,6 +38,8 @@ public class StreamGroupedFold<IN, OUT, KEY>
implements OneInputStreamOperator<IN, OUT>, OutputTypeConfigurable<OUT> {
private static final long serialVersionUID = 1L;
+
+ private static final String STATE_NAME = "_op_state";
// Grouped values
private transient OperatorState<OUT> values;
@@ -68,7 +70,7 @@ public class StreamGroupedFold<IN, OUT, KEY>
new DataInputStream(bais)
);
initialValue = outTypeSerializer.deserialize(in);
- values = createKeyValueState(outTypeSerializer, null);
+ values = createKeyValueState(STATE_NAME, outTypeSerializer, null);
}
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/7c205432/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedReduce.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedReduce.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedReduce.java
index ebc4b09..ae15e92 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedReduce.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedReduce.java
@@ -27,6 +27,8 @@ public class StreamGroupedReduce<IN> extends AbstractUdfStreamOperator<IN, Reduc
implements OneInputStreamOperator<IN, IN> {
private static final long serialVersionUID = 1L;
+
+ private static final String STATE_NAME = "_op_state";
private transient OperatorState<IN> values;
@@ -41,7 +43,7 @@ public class StreamGroupedReduce<IN> extends AbstractUdfStreamOperator<IN, Reduc
@Override
public void open() throws Exception {
super.open();
- values = createKeyValueState(serializer, null);
+ values = createKeyValueState(STATE_NAME, serializer, null);
}
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/7c205432/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java
index a51bb27..87a9abd 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java
@@ -28,6 +28,7 @@ import org.apache.flink.runtime.execution.Environment;
import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
import org.apache.flink.streaming.runtime.operators.Triggerable;
+import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -46,10 +47,10 @@ public class StreamingRuntimeContext extends AbstractRuntimeUDFContext {
private final Environment taskEnvironment;
/** The key/value state, if the user-function requests it */
- private OperatorState<?> keyValueState;
+ private HashMap<String, OperatorState<?>> keyValueStates;
/** Type of the values stored in the state, to make sure repeated requests of the state are consistent */
- private TypeInformation<?> stateTypeInfo;
+ private HashMap<String, TypeInformation<?>> stateTypeInfos;
public StreamingRuntimeContext(AbstractStreamOperator<?> operator,
@@ -107,7 +108,7 @@ public class StreamingRuntimeContext extends AbstractRuntimeUDFContext {
// ------------------------------------------------------------------------
@Override
- public <S> OperatorState<S> getKeyValueState(Class<S> stateType, S defaultState) {
+ public <S> OperatorState<S> getKeyValueState(String name, Class<S> stateType, S defaultState) {
requireNonNull(stateType, "The state type class must not be null");
TypeInformation<S> typeInfo;
@@ -120,35 +121,48 @@ public class StreamingRuntimeContext extends AbstractRuntimeUDFContext {
"Please specify the TypeInformation directly.", e);
}
- return getKeyValueState(typeInfo, defaultState);
+ return getKeyValueState(name, typeInfo, defaultState);
}
@Override
- public <S> OperatorState<S> getKeyValueState(TypeInformation<S> stateType, S defaultState) {
+ public <S> OperatorState<S> getKeyValueState(String name, TypeInformation<S> stateType, S defaultState) {
+ requireNonNull(name, "The name of the state must not be null");
requireNonNull(stateType, "The state type information must not be null");
+ OperatorState<?> previousState;
+
// check if this is a repeated call to access the state
- if (this.stateTypeInfo != null && this.keyValueState != null) {
+ if (this.stateTypeInfos != null && this.keyValueStates != null &&
+ (previousState = this.keyValueStates.get(name)) != null) {
+
// repeated call
- if (this.stateTypeInfo.equals(stateType)) {
+ TypeInformation<?> previousType;
+ if (stateType.equals((previousType = this.stateTypeInfos.get(name)))) {
// valid case, same type requested again
@SuppressWarnings("unchecked")
- OperatorState<S> previous = (OperatorState<S>) this.keyValueState;
+ OperatorState<S> previous = (OperatorState<S>) previousState;
return previous;
}
else {
// invalid case, different type requested this time
throw new IllegalStateException("Cannot initialize key/value state for type " + stateType +
" ; The key/value state has already been created and initialized for a different type: " +
- this.stateTypeInfo);
+ previousType);
}
}
else {
// first time access to the key/value state
+ if (this.stateTypeInfos == null) {
+ this.stateTypeInfos = new HashMap<>();
+ }
+ if (this.keyValueStates == null) {
+ this.keyValueStates = new HashMap<>();
+ }
+
try {
- OperatorState<S> state = operator.createKeyValueState(stateType, defaultState);
- this.keyValueState = state;
- this.stateTypeInfo = stateType;
+ OperatorState<S> state = operator.createKeyValueState(name, stateType, defaultState);
+ this.keyValueStates.put(name, state);
+ this.stateTypeInfos.put(name, stateType);
return state;
}
catch (RuntimeException e) {
http://git-wip-us.apache.org/repos/asf/flink/blob/7c205432/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskState.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskState.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskState.java
index 2fce7af..334fd44 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskState.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskState.java
@@ -22,6 +22,9 @@ import org.apache.flink.runtime.state.StateHandle;
import org.apache.flink.streaming.api.state.KvStateSnapshot;
import java.io.Serializable;
+import java.util.ConcurrentModificationException;
+import java.util.HashMap;
+import java.util.Iterator;
/**
* The state checkpointed by a {@link org.apache.flink.streaming.api.operators.AbstractStreamOperator}.
@@ -40,7 +43,7 @@ public class StreamTaskState implements Serializable {
private StateHandle<Serializable> functionState;
- private KvStateSnapshot<?, ?, ?> kvState;
+ private HashMap<String, KvStateSnapshot<?, ?, ?>> kvStates;
// ------------------------------------------------------------------------
@@ -60,12 +63,12 @@ public class StreamTaskState implements Serializable {
this.functionState = functionState;
}
- public KvStateSnapshot<?, ?, ?> getKvState() {
- return kvState;
+ public HashMap<String, KvStateSnapshot<?, ?, ?>> getKvStates() {
+ return kvStates;
}
- public void setKvState(KvStateSnapshot<?, ?, ?> kvState) {
- this.kvState = kvState;
+ public void setKvStates(HashMap<String, KvStateSnapshot<?, ?, ?>> kvStates) {
+ this.kvStates = kvStates;
}
// ------------------------------------------------------------------------
@@ -77,7 +80,7 @@ public class StreamTaskState implements Serializable {
* @return True, if all state is null, false if at least one state is not null.
*/
public boolean isEmpty() {
- return operatorState == null & functionState == null & kvState == null;
+ return operatorState == null & functionState == null & kvStates == null;
}
/**
@@ -89,7 +92,7 @@ public class StreamTaskState implements Serializable {
public void discardState() throws Exception {
StateHandle<?> operatorState = this.operatorState;
StateHandle<?> functionState = this.functionState;
- KvStateSnapshot<?, ?, ?> kvState = this.kvState;
+ HashMap<String, KvStateSnapshot<?, ?, ?>> kvStates = this.kvStates;
if (operatorState != null) {
operatorState.discardState();
@@ -97,12 +100,25 @@ public class StreamTaskState implements Serializable {
if (functionState != null) {
functionState.discardState();
}
- if (kvState != null) {
- kvState.discardState();
+ if (kvStates != null) {
+ while (kvStates.size() > 0) {
+ try {
+ Iterator<KvStateSnapshot<?, ?, ?>> values = kvStates.values().iterator();
+ while (values.hasNext()) {
+ KvStateSnapshot<?, ?, ?> s = values.next();
+ s.discardState();
+ values.remove();
+ }
+ }
+ catch (ConcurrentModificationException e) {
+ // fall through the loop
+ }
+ }
}
this.operatorState = null;
this.functionState = null;
- this.kvState = null;
+ this.kvStates = null;
}
}
+
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/flink/blob/7c205432/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/keys/KeySelectorUtil.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/keys/KeySelectorUtil.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/keys/KeySelectorUtil.java
index 9c76d95..afbd8ab 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/keys/KeySelectorUtil.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/util/keys/KeySelectorUtil.java
@@ -18,10 +18,13 @@
package org.apache.flink.streaming.util.keys;
import java.lang.reflect.Array;
+import java.util.Arrays;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.functions.InvalidTypesException;
import org.apache.flink.api.common.functions.Partitioner;
+import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
+import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeutils.CompositeType;
import org.apache.flink.api.common.typeutils.TypeComparator;
@@ -31,6 +34,8 @@ import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
import org.apache.flink.api.java.typeutils.TupleTypeInfo;
+import static java.util.Objects.requireNonNull;
+
/**
* Utility class that contains helper methods to manipulating {@link KeySelector} for streaming.
*/
@@ -49,7 +54,7 @@ public final class KeySelectorUtil {
// use ascending order here, the code paths for that are usually a slight bit faster
boolean[] orders = new boolean[numKeyFields];
- TypeInformation[] typeInfos = new TypeInformation[numKeyFields];
+ TypeInformation<?>[] typeInfos = new TypeInformation<?>[numKeyFields];
for (int i = 0; i < numKeyFields; i++) {
orders[i] = true;
typeInfos[i] = compositeType.getTypeAt(logicalKeyPositions[i]);
@@ -59,31 +64,71 @@ public final class KeySelectorUtil {
return new ComparableKeySelector<>(comparator, numKeyFields, new TupleTypeInfo<>(typeInfos));
}
+ public static <X> ArrayKeySelector<X> getSelectorForArray(int[] positions, TypeInformation<X> typeInfo) {
+ if (positions == null || positions.length == 0 || positions.length > Tuple.MAX_ARITY) {
+ throw new IllegalArgumentException("Array keys must have between 1 and " + Tuple.MAX_ARITY + " fields.");
+ }
+
+ TypeInformation<?> componentType;
+
+ if (typeInfo instanceof BasicArrayTypeInfo) {
+ BasicArrayTypeInfo<X, ?> arrayInfo = (BasicArrayTypeInfo<X, ?>) typeInfo;
+ componentType = arrayInfo.getComponentInfo();
+ }
+ else if (typeInfo instanceof PrimitiveArrayTypeInfo) {
+ PrimitiveArrayTypeInfo<X> arrayType = (PrimitiveArrayTypeInfo<X>) typeInfo;
+ componentType = arrayType.getComponentType();
+ }
+ else {
+ throw new IllegalArgumentException("This method only supports arrays of primitives and boxed primitives.");
+ }
+
+ TypeInformation<?>[] primitiveInfos = new TypeInformation<?>[positions.length];
+ Arrays.fill(primitiveInfos, componentType);
+
+ return new ArrayKeySelector<>(positions, new TupleTypeInfo<>(primitiveInfos));
+ }
+
- public static <X, K> KeySelector<X, K> getSelectorForOneKey(Keys<X> keys, Partitioner<K> partitioner, TypeInformation<X> typeInfo,
- ExecutionConfig executionConfig) {
+ public static <X, K> KeySelector<X, K> getSelectorForOneKey(
+ Keys<X> keys, Partitioner<K> partitioner, TypeInformation<X> typeInfo, ExecutionConfig executionConfig)
+ {
+ if (!(typeInfo instanceof CompositeType)) {
+ throw new InvalidTypesException(
+ "This key operation requires a composite type such as Tuples, POJOs, case classes, etc");
+ }
if (partitioner != null) {
keys.validateCustomPartitioner(partitioner, null);
}
+ CompositeType<X> compositeType = (CompositeType<X>) typeInfo;
int[] logicalKeyPositions = keys.computeLogicalKeyPositions();
-
if (logicalKeyPositions.length != 1) {
throw new IllegalArgumentException("There must be exactly 1 key specified");
}
-
- TypeComparator<X> comparator = ((CompositeType<X>) typeInfo).createComparator(
- logicalKeyPositions, new boolean[1], 0, executionConfig);
+
+ TypeComparator<X> comparator = compositeType.createComparator(
+ logicalKeyPositions, new boolean[] { true }, 0, executionConfig);
return new OneKeySelector<>(comparator);
}
+ // ------------------------------------------------------------------------
+
/**
* Private constructor to prevent instantiation.
*/
private KeySelectorUtil() {
throw new RuntimeException();
}
-
+
+ // ------------------------------------------------------------------------
+
+ /**
+ * Key extractor that extracts a single field via a generic comparator.
+ *
+ * @param <IN> The type of the elements where the key is extracted from.
+ * @param <K> The type of the key.
+ */
public static final class OneKeySelector<IN, K> implements KeySelector<IN, K> {
private static final long serialVersionUID = 1L;
@@ -94,8 +139,8 @@ public final class KeySelectorUtil {
* are null), it does not have any serialization problems */
@SuppressWarnings("NonSerializableFieldInSerializableClass")
private final Object[] keyArray;
-
- public OneKeySelector(TypeComparator<IN> comparator) {
+
+ OneKeySelector(TypeComparator<IN> comparator) {
this.comparator = comparator;
this.keyArray = new Object[1];
}
@@ -121,18 +166,18 @@ public final class KeySelectorUtil {
private final TypeComparator<IN> comparator;
private final int keyLength;
- private final TupleTypeInfo tupleTypeInfo;
+ private transient TupleTypeInfo<Tuple> tupleTypeInfo;
/** Reusable array to hold the key objects. Since this is initially empty (all positions
* are null), it does not have any serialization problems */
@SuppressWarnings("NonSerializableFieldInSerializableClass")
private final Object[] keyArray;
- public ComparableKeySelector(TypeComparator<IN> comparator, int keyLength, TupleTypeInfo tupleTypeInfo) {
+ ComparableKeySelector(TypeComparator<IN> comparator, int keyLength, TupleTypeInfo<Tuple> tupleTypeInfo) {
this.comparator = comparator;
this.keyLength = keyLength;
this.tupleTypeInfo = tupleTypeInfo;
- keyArray = new Object[keyLength];
+ this.keyArray = new Object[keyLength];
}
@Override
@@ -147,6 +192,9 @@ public final class KeySelectorUtil {
@Override
public TypeInformation<Tuple> getProducedType() {
+ if (tupleTypeInfo == null) {
+ throw new IllegalStateException("The return type information is not available after serialization");
+ }
return tupleTypeInfo;
}
}
@@ -158,23 +206,35 @@ public final class KeySelectorUtil {
*
* @param <IN> The type from which the key is extracted, i.e., the array type.
*/
- public static final class ArrayKeySelector<IN> implements KeySelector<IN, Tuple> {
+ public static final class ArrayKeySelector<IN> implements KeySelector<IN, Tuple>, ResultTypeQueryable<Tuple> {
private static final long serialVersionUID = 1L;
private final int[] fields;
+ private final Class<? extends Tuple> tupleClass;
+ private transient TupleTypeInfo<Tuple> returnType;
- public ArrayKeySelector(int... fields) {
- this.fields = fields;
+ ArrayKeySelector(int[] fields, TupleTypeInfo<Tuple> returnType) {
+ this.fields = requireNonNull(fields);
+ this.returnType = requireNonNull(returnType);
+ this.tupleClass = Tuple.getTupleClass(fields.length);
}
@Override
public Tuple getKey(IN value) throws Exception {
- Tuple key = Tuple.getTupleClass(fields.length).newInstance();
+ Tuple key = tupleClass.newInstance();
for (int i = 0; i < fields.length; i++) {
key.setField(Array.get(value, fields[i]), i);
}
return key;
}
+
+ @Override
+ public TypeInformation<Tuple> getProducedType() {
+ if (returnType == null) {
+ throw new IllegalStateException("The return type information is not available after serialization");
+ }
+ return returnType;
+ }
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/7c205432/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/ChainedRuntimeContextTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/ChainedRuntimeContextTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/ChainedRuntimeContextTest.java
index fdf7697..68a047c 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/ChainedRuntimeContextTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/ChainedRuntimeContextTest.java
@@ -25,9 +25,8 @@ import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
import org.apache.flink.streaming.util.NoOpSink;
-import org.apache.flink.streaming.util.ReceiveCheckNoOpSink;
import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.apache.flink.streaming.util.TestStreamEnvironment;
+
import org.junit.Test;
@SuppressWarnings("serial")
http://git-wip-us.apache.org/repos/asf/flink/blob/7c205432/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/keys/ArrayKeySelectorTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/keys/ArrayKeySelectorTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/keys/ArrayKeySelectorTest.java
index 9d807cf..63375a7 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/keys/ArrayKeySelectorTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/util/keys/ArrayKeySelectorTest.java
@@ -18,6 +18,8 @@
package org.apache.flink.streaming.util.keys;
+import org.apache.flink.api.common.typeinfo.BasicArrayTypeInfo;
+import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo;
import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.api.java.tuple.Tuple2;
import org.junit.Test;
@@ -29,15 +31,17 @@ public class ArrayKeySelectorTest {
@Test
public void testObjectArrays() {
try {
- Object[] array1 = { "a", "b", "c", "d", "e" };
- Object[] array2 = { "v", "w", "x", "y", "z" };
+ String[] array1 = { "a", "b", "c", "d", "e" };
+ String[] array2 = { "v", "w", "x", "y", "z" };
- KeySelectorUtil.ArrayKeySelector<Object[]> singleFieldSelector = new KeySelectorUtil.ArrayKeySelector<>(1);
+ KeySelectorUtil.ArrayKeySelector<String[]> singleFieldSelector =
+ KeySelectorUtil.getSelectorForArray(new int[] {1}, BasicArrayTypeInfo.STRING_ARRAY_TYPE_INFO);
assertEquals(new Tuple1<>("b"), singleFieldSelector.getKey(array1));
assertEquals(new Tuple1<>("w"), singleFieldSelector.getKey(array2));
- KeySelectorUtil.ArrayKeySelector<Object[]> twoFieldsSelector = new KeySelectorUtil.ArrayKeySelector<>(3, 0);
+ KeySelectorUtil.ArrayKeySelector<String[]> twoFieldsSelector =
+ KeySelectorUtil.getSelectorForArray(new int[] {3, 0}, BasicArrayTypeInfo.STRING_ARRAY_TYPE_INFO);
assertEquals(new Tuple2<>("d", "a"), twoFieldsSelector.getKey(array1));
assertEquals(new Tuple2<>("y", "v"), twoFieldsSelector.getKey(array2));
@@ -55,13 +59,15 @@ public class ArrayKeySelectorTest {
int[] array1 = { 1, 2, 3, 4, 5 };
int[] array2 = { -5, -4, -3, -2, -1, 0 };
- KeySelectorUtil.ArrayKeySelector<int[]> singleFieldSelector = new KeySelectorUtil.ArrayKeySelector<>(1);
+ KeySelectorUtil.ArrayKeySelector<int[]> singleFieldSelector =
+ KeySelectorUtil.getSelectorForArray(new int[] {1}, PrimitiveArrayTypeInfo.INT_PRIMITIVE_ARRAY_TYPE_INFO);
assertEquals(new Tuple1<>(2), singleFieldSelector.getKey(array1));
assertEquals(new Tuple1<>(-4), singleFieldSelector.getKey(array2));
- KeySelectorUtil.ArrayKeySelector<int[]> twoFieldsSelector = new KeySelectorUtil.ArrayKeySelector<>(3, 0);
-
+ KeySelectorUtil.ArrayKeySelector<int[]> twoFieldsSelector =
+ KeySelectorUtil.getSelectorForArray(new int[] {3, 0}, PrimitiveArrayTypeInfo.INT_PRIMITIVE_ARRAY_TYPE_INFO);
+
assertEquals(new Tuple2<>(4, 1), twoFieldsSelector.getKey(array1));
assertEquals(new Tuple2<>(-2, -5), twoFieldsSelector.getKey(array2));
http://git-wip-us.apache.org/repos/asf/flink/blob/7c205432/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedStreams.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedStreams.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedStreams.scala
index f7413b7..3ff773f 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedStreams.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedStreams.scala
@@ -362,4 +362,3 @@ class KeySelectorWithType[IN, K](
override def getProducedType: TypeInformation[K] = info
}
-
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/flink/blob/7c205432/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala
index 84354a3..9f5c069 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/KeyedStream.scala
@@ -36,6 +36,15 @@ import scala.reflect.ClassTag
class KeyedStream[T, K](javaStream: KeyedJavaStream[T, K]) extends DataStream[T](javaStream) {
// ------------------------------------------------------------------------
+ // Properties
+ // ------------------------------------------------------------------------
+
+ /**
+ * Gets the type of the key by which this stream is keyed.
+ */
+ def getKeyType = javaStream.getKeyType()
+
+ // ------------------------------------------------------------------------
// Windowing
// ------------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/7c205432/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/StatefulFunction.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/StatefulFunction.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/StatefulFunction.scala
index 5a591a8..d66cfdb 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/StatefulFunction.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/StatefulFunction.scala
@@ -43,6 +43,6 @@ trait StatefulFunction[I, O, S] extends RichFunction {
}
override def open(c: Configuration) = {
- state = getRuntimeContext().getKeyValueState[S](stateType, null.asInstanceOf[S])
+ state = getRuntimeContext().getKeyValueState[S]("state", stateType, null.asInstanceOf[S])
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/7c205432/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala
index fe85fd1..988e7ec 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/DataStreamTest.scala
@@ -118,7 +118,7 @@ class DataStreamTest extends StreamingMultipleProgramsTestBase {
val partition1: DataStream[_] = src1.partitionByHash(0)
val partition2: DataStream[_] = src1.partitionByHash(1, 0)
val partition3: DataStream[_] = src1.partitionByHash("_1")
- val partition4: DataStream[_] = src1.partitionByHash((x : (Long, Long)) => x._1);
+ val partition4: DataStream[_] = src1.partitionByHash((x : (Long, Long)) => x._1)
val pid1 = createDownStreamId(partition1)
val pid2 = createDownStreamId(partition2)
http://git-wip-us.apache.org/repos/asf/flink/blob/7c205432/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StateTestPrograms.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StateTestPrograms.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StateTestPrograms.scala
index 7904bcb..b2e05b3 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StateTestPrograms.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/StateTestPrograms.scala
@@ -31,11 +31,12 @@ object StateTestPrograms {
// test stateful map
env.generateSequence(0, 10).setParallelism(1)
- .keyBy(x => x)
+ .map { v => (1, v) }.setParallelism(1)
+ .keyBy(_._1)
.mapWithState((in, count: Option[Long]) =>
count match {
- case Some(c) => (in - c, Some(c + 1))
- case None => (in, Some(1L))
+ case Some(c) => (in._2 - c, Some(c + 1))
+ case None => (in._2, Some(1L))
}).setParallelism(1)
.addSink(new RichSinkFunction[Long]() {
@@ -49,12 +50,12 @@ object StateTestPrograms {
})
// test stateful flatmap
- env.fromElements("Fir st-", "Hello world")
- .keyBy(x => x)
+ env.fromElements((1, "First"), (2, "Second"), (1, "Hello world"))
+ .keyBy(_._1)
.flatMapWithState((w, s: Option[String]) =>
s match {
- case Some(state) => (w.split(" ").toList.map(state + _), Some(w))
- case None => (List(w), Some(w))
+ case Some(state) => (w._2.split(" ").toList.map(state + _), Some(w._2))
+ case None => (List(w._2), Some(w._2))
})
.setParallelism(1)
@@ -62,10 +63,11 @@ object StateTestPrograms {
val received = new util.HashSet[String]()
override def invoke(in: String) = { received.add(in) }
override def close() = {
- assert(received.size() == 3)
- assert(received.contains("Fir st-"))
- assert(received.contains("Fir st-Hello"))
- assert(received.contains("Fir st-world"))
+ assert(received.size() == 4)
+ assert(received.contains("First"))
+ assert(received.contains("Second"))
+ assert(received.contains("FirstHello"))
+ assert(received.contains("Firstworld"))
}
}).setParallelism(1)
http://git-wip-us.apache.org/repos/asf/flink/blob/7c205432/flink-tests/src/test/java/org/apache/flink/test/checkpointing/PartitionedStateCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/PartitionedStateCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/PartitionedStateCheckpointingITCase.java
index 0fcedda..67c0189 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/PartitionedStateCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/PartitionedStateCheckpointingITCase.java
@@ -107,7 +107,9 @@ public class PartitionedStateCheckpointingITCase extends StreamFaultToleranceTes
@Override
public void open(Configuration parameters) throws IOException {
step = getRuntimeContext().getNumberOfParallelSubtasks();
- index = getRuntimeContext().getIndexOfThisSubtask();
+ if (index == 0) {
+ index = getRuntimeContext().getIndexOfThisSubtask();
+ }
}
@Override
@@ -165,7 +167,7 @@ public class PartitionedStateCheckpointingITCase extends StreamFaultToleranceTes
failurePos = (new Random().nextLong() % (failurePosMax - failurePosMin)) + failurePosMin;
count = 0;
- sum = getRuntimeContext().getKeyValueState(Long.class, 0L);
+ sum = getRuntimeContext().getKeyValueState("my_state", Long.class, 0L);
}
@Override
@@ -187,17 +189,26 @@ public class PartitionedStateCheckpointingITCase extends StreamFaultToleranceTes
private static Map<Integer, Long> allCounts = new ConcurrentHashMap<Integer, Long>();
- private OperatorState<NonSerializableLong> counts;
+ private OperatorState<NonSerializableLong> aCounts;
+ private OperatorState<Long> bCounts;
@Override
public void open(Configuration parameters) throws IOException {
- counts = getRuntimeContext().getKeyValueState(NonSerializableLong.class, NonSerializableLong.of(0L));
+ aCounts = getRuntimeContext().getKeyValueState(
+ "a", NonSerializableLong.class, NonSerializableLong.of(0L));
+ bCounts = getRuntimeContext().getKeyValueState("b", Long.class, 0L);
}
@Override
public void invoke(Tuple2<Integer, Long> value) throws Exception {
- long currentCount = counts.value().value + 1;
- counts.update(NonSerializableLong.of(currentCount));
+ long ac = aCounts.value().value;
+ long bc = bCounts.value();
+ assertEquals(ac, bc);
+
+ long currentCount = ac + 1;
+ aCounts.update(NonSerializableLong.of(currentCount));
+ bCounts.update(currentCount);
+
allCounts.put(value.f0, currentCount);
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/7c205432/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java
index 992a679..e98696e 100644
--- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java
@@ -252,7 +252,7 @@ public class StreamCheckpointingITCase extends StreamFaultToleranceTestBase {
failurePos = (new Random().nextLong() % (failurePosMax - failurePosMin)) + failurePosMin;
count = 0;
- pCount = getRuntimeContext().getKeyValueState(Long.class, 0L);
+ pCount = getRuntimeContext().getKeyValueState("pCount", Long.class, 0L);
}
@Override
[24/24] flink git commit: [hotfix] [storm compatibility] Deactivate
tests for split stream field grouping,
which do not work in teh runtime and are now caught earlier
Posted by se...@apache.org.
[hotfix] [storm compatibility] Deactivate tests for split stream field grouping, which do not work in teh runtime and are now caught earlier
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/34c232e9
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/34c232e9
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/34c232e9
Branch: refs/heads/master
Commit: 34c232e9bd8400b8409434116d0257cf643a0eaa
Parents: d68c8b1
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Oct 16 16:31:11 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 16 16:31:11 2015 +0200
----------------------------------------------------------------------
.../java/org/apache/flink/storm/api/FlinkTopologyBuilderTest.java | 3 +++
1 file changed, 3 insertions(+)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/34c232e9/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyBuilderTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyBuilderTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyBuilderTest.java
index fa5c8d8..906d081 100644
--- a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyBuilderTest.java
+++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyBuilderTest.java
@@ -21,6 +21,7 @@ import org.apache.flink.storm.util.TestDummyBolt;
import org.apache.flink.storm.util.TestDummySpout;
import org.apache.flink.storm.util.TestSink;
+import org.junit.Ignore;
import org.junit.Test;
import backtype.storm.tuple.Fields;
@@ -53,6 +54,7 @@ public class FlinkTopologyBuilderTest {
}
@Test
+ @Ignore
public void testFieldsGroupingOnMultipleSpoutOutputStreams() {
FlinkTopologyBuilder flinkBuilder = new FlinkTopologyBuilder();
@@ -64,6 +66,7 @@ public class FlinkTopologyBuilderTest {
}
@Test
+ @Ignore
public void testFieldsGroupingOnMultipleBoltOutputStreams() {
FlinkTopologyBuilder flinkBuilder = new FlinkTopologyBuilder();
[20/24] flink git commit: [hotfix] [streaming] Remove obsolete
internal state handle classes
Posted by se...@apache.org.
[hotfix] [streaming] Remove obsolete internal state handle classes
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/d68c8b1d
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/d68c8b1d
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/d68c8b1d
Branch: refs/heads/master
Commit: d68c8b1d59493923bc980ccadee30afcab1dfd35
Parents: 7c20543
Author: Stephan Ewen <se...@apache.org>
Authored: Sat Oct 10 03:15:15 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 16 15:26:11 2015 +0200
----------------------------------------------------------------------
.../runtime/state/ByteStreamStateHandle.java | 100 ---------------
.../flink/runtime/state/FileStateHandle.java | 70 -----------
.../runtime/state/PartitionedStateHandle.java | 53 --------
.../state/ByteStreamStateHandleTest.java | 126 -------------------
4 files changed, 349 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/d68c8b1d/flink-runtime/src/main/java/org/apache/flink/runtime/state/ByteStreamStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/ByteStreamStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ByteStreamStateHandle.java
deleted file mode 100644
index 7ecfe62..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/ByteStreamStateHandle.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * 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.runtime.state;
-
-import org.apache.flink.util.InstantiationUtil;
-
-import java.io.InputStream;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-
-/**
- * Statehandle that writes/reads the contents of the serializable checkpointed
- * state to the provided input and outputstreams using default java
- * serialization.
- *
- */
-public abstract class ByteStreamStateHandle implements StateHandle<Serializable> {
-
- private static final long serialVersionUID = -962025800339325828L;
-
- private transient Serializable state;
- private boolean isWritten = false;
-
- public ByteStreamStateHandle(Serializable state) {
- if (state != null) {
- this.state = state;
- } else {
- throw new RuntimeException("State cannot be null");
- }
- }
-
- /**
- * The state will be written to the stream returned by this method.
- */
- protected abstract OutputStream getOutputStream() throws Exception;
-
- /**
- * The state will be read from the stream returned by this method.
- */
- protected abstract InputStream getInputStream() throws Exception;
-
- @Override
- public Serializable getState(ClassLoader userCodeClassLoader) throws Exception {
- if (!stateFetched()) {
- ObjectInputStream stream = new InstantiationUtil.ClassLoaderObjectInputStream(getInputStream(), userCodeClassLoader);
- try {
- state = (Serializable) stream.readObject();
- } finally {
- stream.close();
- }
- }
- return state;
- }
-
- private void writeObject(ObjectOutputStream oos) throws Exception {
- if (!isWritten) {
- ObjectOutputStream stream = new ObjectOutputStream(getOutputStream());
- try {
- stream.writeObject(state);
- isWritten = true;
- } finally {
- stream.close();
- }
- }
- oos.defaultWriteObject();
- }
-
- /**
- * Checks whether the state has already been fetched from the remote
- * storage.
- */
- public boolean stateFetched() {
- return state != null;
- }
-
- /**
- * Checks whether the state has already been written to the external store
- */
- public boolean isWritten() {
- return isWritten;
- }
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/d68c8b1d/flink-runtime/src/main/java/org/apache/flink/runtime/state/FileStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/FileStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/FileStateHandle.java
deleted file mode 100644
index c45990b..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/FileStateHandle.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * 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.runtime.state;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-import java.net.URI;
-import java.net.URISyntaxException;
-
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.util.StringUtils;
-
-import java.util.Random;
-
-/**
- * Statehandle that writes the checkpointed state to a random file in the
- * provided checkpoint directory. Any Flink supported File system can be used
- * but it is advised to use a filesystem that is persistent in case of node
- * failures, such as HDFS or Tachyon.
- *
- */
-public class FileStateHandle extends ByteStreamStateHandle {
-
- private static final long serialVersionUID = 1L;
-
- private String pathString;
-
- public FileStateHandle(Serializable state, String folder) {
- super(state);
- this.pathString = folder + "/" + randomString();
- }
-
- protected OutputStream getOutputStream() throws IOException, URISyntaxException {
- return FileSystem.get(new URI(pathString)).create(new Path(pathString), true);
- }
-
- protected InputStream getInputStream() throws IOException, URISyntaxException {
- return FileSystem.get(new URI(pathString)).open(new Path(pathString));
- }
-
- private String randomString() {
- final byte[] bytes = new byte[20];
- new Random().nextBytes(bytes);
- return StringUtils.byteToHexString(bytes);
- }
-
- @Override
- public void discardState() throws Exception {
- FileSystem.get(new URI(pathString)).delete(new Path(pathString), false);
- }
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/d68c8b1d/flink-runtime/src/main/java/org/apache/flink/runtime/state/PartitionedStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/PartitionedStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/PartitionedStateHandle.java
deleted file mode 100644
index 9ec748b..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/PartitionedStateHandle.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * 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.runtime.state;
-
-import java.io.Serializable;
-import java.util.Map;
-
-/**
- * Wrapper for storing the handles for each state in a partitioned form. It can
- * be used to repartition the state before re-injecting to the tasks.
- *
- * TODO: This class needs testing!
- */
-public class PartitionedStateHandle implements
- StateHandle<Map<Serializable, StateHandle<Serializable>>> {
-
- private static final long serialVersionUID = 7505365403501402100L;
-
- Map<Serializable, StateHandle<Serializable>> handles;
-
- public PartitionedStateHandle(Map<Serializable, StateHandle<Serializable>> handles) {
- this.handles = handles;
- }
-
- @Override
- public Map<Serializable, StateHandle<Serializable>> getState(ClassLoader userCodeClassLoader) throws Exception {
- return handles;
- }
-
- @Override
- public void discardState() throws Exception {
- for (StateHandle<Serializable> handle : handles.values()) {
- handle.discardState();
- }
- }
-
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/d68c8b1d/flink-runtime/src/test/java/org/apache/flink/runtime/state/ByteStreamStateHandleTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/ByteStreamStateHandleTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/ByteStreamStateHandleTest.java
deleted file mode 100644
index c667139..0000000
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/ByteStreamStateHandleTest.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/*
- * 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.runtime.state;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.Serializable;
-
-import org.apache.flink.util.InstantiationUtil;
-import org.junit.Test;
-
-public class ByteStreamStateHandleTest {
-
- @Test
- public void testHandle() throws Exception {
- final ClassLoader cl = this.getClass().getClassLoader();
- MockHandle handle;
-
- try {
- handle = new MockHandle(null);
- fail();
- } catch (RuntimeException e) {
- // expected behaviour
- }
-
- handle = new MockHandle(1);
-
- assertEquals(1, handle.getState(cl));
- assertTrue(handle.stateFetched());
- assertFalse(handle.isWritten());
- assertFalse(handle.discarded);
-
- MockHandle handleDs = serializeDeserialize(handle);
-
- assertEquals(1, handle.getState(cl));
- assertTrue(handle.stateFetched());
- assertTrue(handle.isWritten());
- assertTrue(handle.generatedOutput);
- assertFalse(handle.discarded);
-
- assertFalse(handleDs.stateFetched());
- assertTrue(handleDs.isWritten());
- assertFalse(handleDs.generatedOutput);
- assertFalse(handle.discarded);
-
- try {
- handleDs.getState(cl);
- fail();
- } catch (UnsupportedOperationException e) {
- // good
- }
-
- MockHandle handleDs2 = serializeDeserialize(handleDs);
-
- assertFalse(handleDs2.stateFetched());
- assertTrue(handleDs2.isWritten());
- assertFalse(handleDs.generatedOutput);
- assertFalse(handleDs2.generatedOutput);
- assertFalse(handleDs2.discarded);
-
- handleDs2.discardState();
- assertTrue(handleDs2.discarded);
-
- }
-
- @SuppressWarnings("unchecked")
- private <X extends StateHandle<?>> X serializeDeserialize(X handle) throws IOException,
- ClassNotFoundException {
- byte[] serialized = InstantiationUtil.serializeObject(handle);
- return (X) InstantiationUtil.deserializeObject(serialized, Thread.currentThread()
- .getContextClassLoader());
- }
-
- private static class MockHandle extends ByteStreamStateHandle {
-
- private static final long serialVersionUID = 1L;
-
- public MockHandle(Serializable state) {
- super(state);
- }
-
- boolean discarded = false;
- transient boolean generatedOutput = false;
-
- @Override
- public void discardState() throws Exception {
- discarded = true;
- }
-
- @Override
- protected OutputStream getOutputStream() throws Exception {
- generatedOutput = true;
- return new ByteArrayOutputStream();
- }
-
- @Override
- protected InputStream getInputStream() throws Exception {
- throw new UnsupportedOperationException();
- }
-
- }
-
-}
[13/24] flink git commit: [FLINK-2550] [streaming] Make fast-path
processing time windows fault tolerant
Posted by se...@apache.org.
[FLINK-2550] [streaming] Make fast-path processing time windows fault tolerant
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/c24dca50
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/c24dca50
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/c24dca50
Branch: refs/heads/master
Commit: c24dca501e1b0e1dcdc38d2e81e0a182bc2ae6bb
Parents: 479bec0
Author: Stephan Ewen <se...@apache.org>
Authored: Thu Oct 8 21:38:39 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 16 15:26:10 2015 +0200
----------------------------------------------------------------------
.../core/memory/DataInputViewStreamWrapper.java | 38 +++
.../memory/DataOutputViewStreamWrapper.java | 59 ++++
.../OutputViewDataOutputStreamWrapper.java | 5 +-
.../OutputViewObjectOutputStreamWrapper.java | 7 +-
.../streaming/api/datastream/KeyedStream.java | 31 +-
.../api/datastream/WindowedStream.java | 21 +-
.../flink/streaming/api/graph/StreamConfig.java | 2 +-
.../flink/streaming/api/graph/StreamGraph.java | 4 +-
.../api/operators/AbstractStreamOperator.java | 12 +-
.../flink/streaming/api/state/StateBackend.java | 81 ++++-
.../api/state/filesystem/FsStateBackend.java | 3 +
.../api/state/memory/MemoryStateBackend.java | 3 +
...ractAlignedProcessingTimeWindowOperator.java | 126 +++++++-
.../windowing/AbstractKeyedTimePanes.java | 82 ++++-
.../windowing/AccumulatingKeyedTimePanes.java | 6 +-
...ccumulatingProcessingTimeWindowOperator.java | 117 ++++++-
.../windowing/AggregatingKeyedTimePanes.java | 6 +-
...AggregatingProcessingTimeWindowOperator.java | 7 +-
.../streaming/runtime/tasks/StreamTask.java | 25 +-
.../streaming/api/AggregationFunctionTest.java | 95 ++++--
.../streaming/api/graph/SlotAllocationTest.java | 21 +-
.../api/operators/StreamGroupedFoldTest.java | 30 +-
.../api/operators/StreamGroupedReduceTest.java | 2 +
...AlignedProcessingTimeWindowOperatorTest.java | 313 ++++++++++++++++--
...AlignedProcessingTimeWindowOperatorTest.java | 321 +++++++++++++++++--
.../flink/streaming/util/MockContext.java | 34 +-
.../util/OneInputStreamOperatorTestHarness.java | 25 +-
27 files changed, 1271 insertions(+), 205 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/c24dca50/flink-core/src/main/java/org/apache/flink/core/memory/DataInputViewStreamWrapper.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/DataInputViewStreamWrapper.java b/flink-core/src/main/java/org/apache/flink/core/memory/DataInputViewStreamWrapper.java
new file mode 100644
index 0000000..80affea
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/core/memory/DataInputViewStreamWrapper.java
@@ -0,0 +1,38 @@
+/*
+ * 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.core.memory;
+
+import java.io.DataInputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+
+public class DataInputViewStreamWrapper extends DataInputStream implements DataInputView {
+
+ public DataInputViewStreamWrapper(InputStream in) {
+ super(in);
+ }
+
+ @Override
+ public void skipBytesToRead(int numBytes) throws IOException {
+ if (skipBytes(numBytes) != numBytes){
+ throw new EOFException("Could not skip " + numBytes + " bytes.");
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/c24dca50/flink-core/src/main/java/org/apache/flink/core/memory/DataOutputViewStreamWrapper.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/DataOutputViewStreamWrapper.java b/flink-core/src/main/java/org/apache/flink/core/memory/DataOutputViewStreamWrapper.java
new file mode 100644
index 0000000..efcc17e
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/core/memory/DataOutputViewStreamWrapper.java
@@ -0,0 +1,59 @@
+/*
+ * 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.core.memory;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+
+public class DataOutputViewStreamWrapper extends DataOutputStream implements DataOutputView {
+
+ private byte[] tempBuffer;
+
+ public DataOutputViewStreamWrapper(OutputStream out) {
+ super(out);
+ }
+
+ @Override
+ public void skipBytesToWrite(int numBytes) throws IOException {
+ if (tempBuffer == null) {
+ tempBuffer = new byte[4096];
+ }
+
+ while (numBytes > 0) {
+ int toWrite = Math.min(numBytes, tempBuffer.length);
+ write(tempBuffer, 0, toWrite);
+ numBytes -= toWrite;
+ }
+ }
+
+ @Override
+ public void write(DataInputView source, int numBytes) throws IOException {
+ if (tempBuffer == null) {
+ tempBuffer = new byte[4096];
+ }
+
+ while (numBytes > 0) {
+ int toCopy = Math.min(numBytes, tempBuffer.length);
+ source.read(tempBuffer, 0, toCopy);
+ write(tempBuffer, 0, toCopy);
+ numBytes -= toCopy;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/c24dca50/flink-core/src/main/java/org/apache/flink/core/memory/OutputViewDataOutputStreamWrapper.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/OutputViewDataOutputStreamWrapper.java b/flink-core/src/main/java/org/apache/flink/core/memory/OutputViewDataOutputStreamWrapper.java
index ffe36c0..3be5d8b 100644
--- a/flink-core/src/main/java/org/apache/flink/core/memory/OutputViewDataOutputStreamWrapper.java
+++ b/flink-core/src/main/java/org/apache/flink/core/memory/OutputViewDataOutputStreamWrapper.java
@@ -21,7 +21,6 @@ package org.apache.flink.core.memory;
import java.io.Closeable;
import java.io.DataOutputStream;
import java.io.IOException;
-import java.util.Arrays;
public class OutputViewDataOutputStreamWrapper implements DataOutputView, Closeable {
@@ -43,9 +42,7 @@ public class OutputViewDataOutputStreamWrapper implements DataOutputView, Closea
@Override
public void skipBytesToWrite(int numBytes) throws IOException {
- byte[] bytes = new byte[numBytes];
- Arrays.fill(bytes, (byte)0);
- out.write(bytes);
+ out.write(new byte[numBytes]);
}
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/c24dca50/flink-core/src/main/java/org/apache/flink/core/memory/OutputViewObjectOutputStreamWrapper.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/OutputViewObjectOutputStreamWrapper.java b/flink-core/src/main/java/org/apache/flink/core/memory/OutputViewObjectOutputStreamWrapper.java
index b84e07e..49cc3a7 100644
--- a/flink-core/src/main/java/org/apache/flink/core/memory/OutputViewObjectOutputStreamWrapper.java
+++ b/flink-core/src/main/java/org/apache/flink/core/memory/OutputViewObjectOutputStreamWrapper.java
@@ -20,9 +20,9 @@ package org.apache.flink.core.memory;
import java.io.IOException;
import java.io.ObjectOutputStream;
-import java.util.Arrays;
public class OutputViewObjectOutputStreamWrapper implements DataOutputView {
+
private final ObjectOutputStream out;
public OutputViewObjectOutputStreamWrapper(ObjectOutputStream out){
@@ -31,10 +31,7 @@ public class OutputViewObjectOutputStreamWrapper implements DataOutputView {
@Override
public void skipBytesToWrite(int numBytes) throws IOException {
- byte[] buffer = new byte[numBytes];
- Arrays.fill(buffer, (byte) 0);
-
- out.write(buffer);
+ out.write(new byte[numBytes]);
}
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/c24dca50/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java
index cdea910..d4a3a77 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java
@@ -60,10 +60,12 @@ import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
* @param <KEY> The type of the key in the Keyed Stream.
*/
public class KeyedStream<T, KEY> extends DataStream<T> {
-
- protected final KeySelector<T, KEY> keySelector;
- protected final TypeInformation<KEY> keyType;
+ /** The key selector that can get the key by which the stream if partitioned from the elements */
+ private final KeySelector<T, KEY> keySelector;
+
+ /** The type of the key by which the stream is partitioned */
+ private final TypeInformation<KEY> keyType;
/**
* Creates a new {@link KeyedStream} using the given {@link KeySelector}
@@ -93,18 +95,35 @@ public class KeyedStream<T, KEY> extends DataStream<T> {
this.keySelector = keySelector;
this.keyType = keyType;
}
-
+ // ------------------------------------------------------------------------
+ // properties
+ // ------------------------------------------------------------------------
+
+ /**
+ * Gets the key selector that can get the key by which the stream if partitioned from the elements.
+ * @return The key selector for the key.
+ */
public KeySelector<T, KEY> getKeySelector() {
return this.keySelector;
}
-
+ /**
+ * Gets the type of the key by which the stream is partitioned.
+ * @return The type of the key by which the stream is partitioned.
+ */
+ public TypeInformation<KEY> getKeyType() {
+ return keyType;
+ }
+
@Override
protected DataStream<T> setConnectionType(StreamPartitioner<T> partitioner) {
throw new UnsupportedOperationException("Cannot override partitioning for KeyedStream.");
}
+ // ------------------------------------------------------------------------
+ // basic transformations
+ // ------------------------------------------------------------------------
@Override
public <R> SingleOutputStreamOperator<R, ?> transform(String operatorName,
@@ -119,8 +138,6 @@ public class KeyedStream<T, KEY> extends DataStream<T> {
return returnStream;
}
-
-
@Override
public DataStreamSink<T> addSink(SinkFunction<T> sinkFunction) {
http://git-wip-us.apache.org/repos/asf/flink/blob/c24dca50/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
index 1b511d8..f1220de 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/datastream/WindowedStream.java
@@ -503,7 +503,10 @@ public class WindowedStream<T, K, W extends Window> {
@SuppressWarnings("unchecked")
OneInputStreamOperator<T, R> op = (OneInputStreamOperator<T, R>)
new AggregatingProcessingTimeWindowOperator<>(
- reducer, input.getKeySelector(), windowLength, windowSlide);
+ reducer, input.getKeySelector(),
+ input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
+ input.getType().createSerializer(getExecutionEnvironment().getConfig()),
+ windowLength, windowSlide);
return input.transform(opName, resultType, op);
}
else if (function instanceof WindowFunction) {
@@ -511,7 +514,10 @@ public class WindowedStream<T, K, W extends Window> {
WindowFunction<T, R, K, TimeWindow> wf = (WindowFunction<T, R, K, TimeWindow>) function;
OneInputStreamOperator<T, R> op = new AccumulatingProcessingTimeWindowOperator<>(
- wf, input.getKeySelector(), windowLength, windowSlide);
+ wf, input.getKeySelector(),
+ input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
+ input.getType().createSerializer(getExecutionEnvironment().getConfig()),
+ windowLength, windowSlide);
return input.transform(opName, resultType, op);
}
} else if (windowAssigner instanceof TumblingTimeWindows && trigger instanceof ProcessingTimeTrigger && evictor == null) {
@@ -528,7 +534,11 @@ public class WindowedStream<T, K, W extends Window> {
@SuppressWarnings("unchecked")
OneInputStreamOperator<T, R> op = (OneInputStreamOperator<T, R>)
new AggregatingProcessingTimeWindowOperator<>(
- reducer, input.getKeySelector(), windowLength, windowSlide);
+ reducer,
+ input.getKeySelector(),
+ input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
+ input.getType().createSerializer(getExecutionEnvironment().getConfig()),
+ windowLength, windowSlide);
return input.transform(opName, resultType, op);
}
else if (function instanceof WindowFunction) {
@@ -536,7 +546,10 @@ public class WindowedStream<T, K, W extends Window> {
WindowFunction<T, R, K, TimeWindow> wf = (WindowFunction<T, R, K, TimeWindow>) function;
OneInputStreamOperator<T, R> op = new AccumulatingProcessingTimeWindowOperator<>(
- wf, input.getKeySelector(), windowLength, windowSlide);
+ wf, input.getKeySelector(),
+ input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
+ input.getType().createSerializer(getExecutionEnvironment().getConfig()),
+ windowLength, windowSlide);
return input.transform(opName, resultType, op);
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/c24dca50/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java
index 55afc93..76be598 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamConfig.java
@@ -386,7 +386,7 @@ public class StreamConfig implements Serializable {
}
}
- public void setStatePartitioner(KeySelector<?, Serializable> partitioner) {
+ public void setStatePartitioner(KeySelector<?, ?> partitioner) {
try {
InstantiationUtil.writeObjectToConfig(partitioner, this.config, STATE_PARTITIONER);
} catch (IOException e) {
http://git-wip-us.apache.org/repos/asf/flink/blob/c24dca50/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
index 4c5c19c..0652406 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/graph/StreamGraph.java
@@ -64,8 +64,6 @@ import org.apache.sling.commons.json.JSONException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import static java.util.Objects.requireNonNull;
-
/**
* Class representing the streaming topology. It contains all the information
* necessary to build the jobgraph for the execution.
@@ -149,7 +147,7 @@ public class StreamGraph extends StreamingPlan {
}
public void setStateBackend(StateBackend<?> backend) {
- this.stateBackend = requireNonNull(backend);
+ this.stateBackend = backend;
}
public StateBackend<?> getStateBackend() {
http://git-wip-us.apache.org/repos/asf/flink/blob/c24dca50/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
index e99d54d..ca86627 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java
@@ -144,13 +144,9 @@ public abstract class AbstractStreamOperator<OUT>
@Override
public StreamTaskState snapshotOperatorState(long checkpointId, long timestamp) throws Exception {
- // here, we deal with operator checkpoints and key/value state snapshots
+ // here, we deal with key/value state snapshots
StreamTaskState state = new StreamTaskState();
-
- // (1) checkpoint the operator, if the operator is stateful
-
- // (2) draw a snapshot of the key/value state
if (keyValueState != null) {
KvStateSnapshot<?, ?, ?> snapshot = keyValueState.shapshot(checkpointId, timestamp);
state.setKvState(snapshot);
@@ -161,10 +157,8 @@ public abstract class AbstractStreamOperator<OUT>
@Override
public void restoreState(StreamTaskState state) throws Exception {
- // (1) checkpoint the operator, if the operator is stateful
-
- // (2) restore the key/value state. the actual restore happens lazily, when the function requests
- // the state again, because the restore method needs information provided by the user function
+ // restore the key/value state. the actual restore happens lazily, when the function requests
+ // the state again, because the restore method needs information provided by the user function
keyValueStateSnapshot = state.getKvState();
}
http://git-wip-us.apache.org/repos/asf/flink/blob/c24dca50/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StateBackend.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StateBackend.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StateBackend.java
index b4fce7e..f4391ad 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StateBackend.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StateBackend.java
@@ -20,6 +20,10 @@ package org.apache.flink.streaming.api.state;
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
import org.apache.flink.runtime.state.StateHandle;
import java.io.IOException;
@@ -57,6 +61,14 @@ public abstract class StateBackend<Backend extends StateBackend<Backend>> implem
* @throws Exception Exceptions may occur during disposal of the state and should be forwarded.
*/
public abstract void disposeAllStateForCurrentJob() throws Exception;
+
+ /**
+ * Closes the state backend, releasing all internal resources, but does not delete any persistent
+ * checkpoint data.
+ *
+ * @throws Exception Exceptions can be forwarded and will be logged by the system
+ */
+ public abstract void close() throws Exception;
// ------------------------------------------------------------------------
// key/value state
@@ -96,7 +108,21 @@ public abstract class StateBackend<Backend extends StateBackend<Backend>> implem
*/
public abstract CheckpointStateOutputStream createCheckpointStateOutputStream(
long checkpointID, long timestamp) throws Exception;
-
+
+ /**
+ * Creates a {@link DataOutputView} stream that writes into the state of the given checkpoint.
+ * When the stream is closes, it returns a state handle that can retrieve the state back.
+ *
+ * @param checkpointID The ID of the checkpoint.
+ * @param timestamp The timestamp of the checkpoint.
+ * @return An DataOutputView stream that writes state for the given checkpoint.
+ *
+ * @throws Exception Exceptions may occur while creating the stream and should be forwarded.
+ */
+ public CheckpointStateOutputView createCheckpointStateOutputView(
+ long checkpointID, long timestamp) throws Exception {
+ return new CheckpointStateOutputView(createCheckpointStateOutputStream(checkpointID, timestamp));
+ }
/**
* Writes the given state into the checkpoint, and returns a handle that can retrieve the state back.
@@ -132,4 +158,57 @@ public abstract class StateBackend<Backend extends StateBackend<Backend>> implem
*/
public abstract StreamStateHandle closeAndGetHandle() throws IOException;
}
+
+ /**
+ * A dedicated DataOutputView stream that produces a {@code StateHandle<DataInputView>} when closed.
+ */
+ public static final class CheckpointStateOutputView extends DataOutputViewStreamWrapper {
+
+ private final CheckpointStateOutputStream out;
+
+ public CheckpointStateOutputView(CheckpointStateOutputStream out) {
+ super(out);
+ this.out = out;
+ }
+
+ /**
+ * Closes the stream and gets a state handle that can create a DataInputView.
+ * producing the data written to this stream.
+ *
+ * @return A state handle that can create an input stream producing the data written to this stream.
+ * @throws IOException Thrown, if the stream cannot be closed.
+ */
+ public StateHandle<DataInputView> closeAndGetHandle() throws IOException {
+ return new DataInputViewHandle(out.closeAndGetHandle());
+ }
+
+ @Override
+ public void close() throws IOException {
+ out.close();
+ }
+ }
+
+ /**
+ * Simple state handle that resolved a {@link DataInputView} from a StreamStateHandle.
+ */
+ private static final class DataInputViewHandle implements StateHandle<DataInputView> {
+
+ private static final long serialVersionUID = 2891559813513532079L;
+
+ private final StreamStateHandle stream;
+
+ private DataInputViewHandle(StreamStateHandle stream) {
+ this.stream = stream;
+ }
+
+ @Override
+ public DataInputView getState(ClassLoader userCodeClassLoader) throws Exception {
+ return new DataInputViewStreamWrapper(stream.getState(userCodeClassLoader));
+ }
+
+ @Override
+ public void discardState() throws Exception {
+ stream.discardState();
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/c24dca50/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsStateBackend.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsStateBackend.java
index 1fc2457..3cbd227 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsStateBackend.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/filesystem/FsStateBackend.java
@@ -231,6 +231,9 @@ public class FsStateBackend extends StateBackend<FsStateBackend> {
}
}
+ @Override
+ public void close() throws Exception {}
+
// ------------------------------------------------------------------------
// state backend operations
// ------------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/c24dca50/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/MemoryStateBackend.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/MemoryStateBackend.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/MemoryStateBackend.java
index b2dfae8..05368bd 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/MemoryStateBackend.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/memory/MemoryStateBackend.java
@@ -75,6 +75,9 @@ public class MemoryStateBackend extends StateBackend<MemoryStateBackend> {
// nothing to do here, GC will do it
}
+ @Override
+ public void close() throws Exception {}
+
// ------------------------------------------------------------------------
// State backend operations
// ------------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/c24dca50/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractAlignedProcessingTimeWindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractAlignedProcessingTimeWindowOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractAlignedProcessingTimeWindowOperator.java
index 227de49..cf8575e 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractAlignedProcessingTimeWindowOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractAlignedProcessingTimeWindowOperator.java
@@ -21,18 +21,24 @@ package org.apache.flink.streaming.runtime.operators.windowing;
import org.apache.commons.math3.util.ArithmeticUtils;
import org.apache.flink.api.common.functions.Function;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.runtime.state.StateHandle;
import org.apache.flink.runtime.util.MathUtils;
import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.operators.TimestampedCollector;
+import org.apache.flink.streaming.api.state.StateBackend;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
import org.apache.flink.streaming.runtime.operators.Triggerable;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.streaming.runtime.tasks.StreamTaskState;
+import static java.util.Objects.requireNonNull;
-public abstract class AbstractAlignedProcessingTimeWindowOperator<KEY, IN, OUT, F extends Function>
+public abstract class AbstractAlignedProcessingTimeWindowOperator<KEY, IN, OUT, STATE, F extends Function>
extends AbstractUdfStreamOperator<OUT, F>
implements OneInputStreamOperator<IN, OUT>, Triggerable {
@@ -45,6 +51,9 @@ public abstract class AbstractAlignedProcessingTimeWindowOperator<KEY, IN, OUT,
private final Function function;
private final KeySelector<IN, KEY> keySelector;
+ private final TypeSerializer<KEY> keySerializer;
+ private final TypeSerializer<STATE> stateTypeSerializer;
+
private final long windowSize;
private final long windowSlide;
private final long paneSize;
@@ -52,24 +61,25 @@ public abstract class AbstractAlignedProcessingTimeWindowOperator<KEY, IN, OUT,
// ----- fields for operator functionality -----
- private transient AbstractKeyedTimePanes<IN, KEY, ?, OUT> panes;
+ private transient AbstractKeyedTimePanes<IN, KEY, STATE, OUT> panes;
private transient TimestampedCollector<OUT> out;
+ private transient RestoredState<IN, KEY, STATE, OUT> restoredState;
+
private transient long nextEvaluationTime;
private transient long nextSlideTime;
protected AbstractAlignedProcessingTimeWindowOperator(
F function,
KeySelector<IN, KEY> keySelector,
+ TypeSerializer<KEY> keySerializer,
+ TypeSerializer<STATE> stateTypeSerializer,
long windowLength,
long windowSlide)
{
super(function);
- if (function == null || keySelector == null) {
- throw new NullPointerException();
- }
if (windowLength < MIN_SLIDE_TIME) {
throw new IllegalArgumentException("Window length must be at least " + MIN_SLIDE_TIME + " msecs");
}
@@ -87,8 +97,10 @@ public abstract class AbstractAlignedProcessingTimeWindowOperator<KEY, IN, OUT,
"The unit of grouping is too small: %d msecs", windowLength, windowSlide, paneSlide));
}
- this.function = function;
- this.keySelector = keySelector;
+ this.function = requireNonNull(function);
+ this.keySelector = requireNonNull(keySelector);
+ this.keySerializer = requireNonNull(keySerializer);
+ this.stateTypeSerializer = requireNonNull(stateTypeSerializer);
this.windowSize = windowLength;
this.windowSlide = windowSlide;
this.paneSize = paneSlide;
@@ -96,7 +108,7 @@ public abstract class AbstractAlignedProcessingTimeWindowOperator<KEY, IN, OUT,
}
- protected abstract AbstractKeyedTimePanes<IN, KEY, ?, OUT> createPanes(
+ protected abstract AbstractKeyedTimePanes<IN, KEY, STATE, OUT> createPanes(
KeySelector<IN, KEY> keySelector, Function function);
// ------------------------------------------------------------------------
@@ -106,19 +118,53 @@ public abstract class AbstractAlignedProcessingTimeWindowOperator<KEY, IN, OUT,
@Override
public void open() throws Exception {
super.open();
-
+
out = new TimestampedCollector<>(output);
- // create the panes that gather the elements per slide
- panes = createPanes(keySelector, function);
-
// decide when to first compute the window and when to slide it
// the values should align with the start of time (that is, the UNIX epoch, not the big bang)
final long now = System.currentTimeMillis();
nextEvaluationTime = now + windowSlide - (now % windowSlide);
nextSlideTime = now + paneSize - (now % paneSize);
+
+ final long firstTriggerTime = Math.min(nextEvaluationTime, nextSlideTime);
+
+ // check if we restored state and if we need to fire some windows based on that restored state
+ if (restoredState == null) {
+ // initial empty state: create empty panes that gather the elements per slide
+ panes = createPanes(keySelector, function);
+ }
+ else {
+ // restored state
+ panes = restoredState.panes;
+
+ long nextPastEvaluationTime = restoredState.nextEvaluationTime;
+ long nextPastSlideTime = restoredState.nextSlideTime;
+ long nextPastTriggerTime = Math.min(nextPastEvaluationTime, nextPastSlideTime);
+ int numPanesRestored = panes.getNumPanes();
+
+ // fire windows from the past as long as there are more panes with data and as long
+ // as the missed trigger times have not caught up with the presence
+ while (numPanesRestored > 0 && nextPastTriggerTime < firstTriggerTime) {
+ // evaluate the window from the past
+ if (nextPastTriggerTime == nextPastEvaluationTime) {
+ computeWindow(nextPastTriggerTime);
+ nextPastEvaluationTime += windowSlide;
+ }
+
+ // evaluate slide from the past
+ if (nextPastTriggerTime == nextPastSlideTime) {
+ panes.slidePanes(numPanesPerWindow);
+ numPanesRestored--;
+ nextPastSlideTime += paneSize;
+ }
+
+ nextPastTriggerTime = Math.min(nextPastEvaluationTime, nextPastSlideTime);
+ }
+ }
- registerTimer(Math.min(nextEvaluationTime, nextSlideTime), this);
+ // make sure the first window happens
+ registerTimer(firstTriggerTime, this);
}
@Override
@@ -197,6 +243,44 @@ public abstract class AbstractAlignedProcessingTimeWindowOperator<KEY, IN, OUT,
}
// ------------------------------------------------------------------------
+ // Checkpointing
+ // ------------------------------------------------------------------------
+
+ @Override
+ public StreamTaskState snapshotOperatorState(long checkpointId, long timestamp) throws Exception {
+ StreamTaskState taskState = super.snapshotOperatorState(checkpointId, timestamp);
+
+ // we write the panes with the key/value maps into the stream, as well as when this state
+ // should have triggered and slided
+ StateBackend.CheckpointStateOutputView out =
+ getStateBackend().createCheckpointStateOutputView(checkpointId, timestamp);
+
+ out.writeLong(nextEvaluationTime);
+ out.writeLong(nextSlideTime);
+ panes.writeToOutput(out, keySerializer, stateTypeSerializer);
+
+ taskState.setOperatorState(out.closeAndGetHandle());
+ return taskState;
+ }
+
+ @Override
+ public void restoreState(StreamTaskState taskState) throws Exception {
+ super.restoreState(taskState);
+
+ @SuppressWarnings("unchecked")
+ StateHandle<DataInputView> inputState = (StateHandle<DataInputView>) taskState.getOperatorState();
+ DataInputView in = inputState.getState(getUserCodeClassloader());
+
+ final long nextEvaluationTime = in.readLong();
+ final long nextSlideTime = in.readLong();
+
+ AbstractKeyedTimePanes<IN, KEY, STATE, OUT> panes = createPanes(keySelector, function);
+ panes.readFromInput(in, keySerializer, stateTypeSerializer);
+
+ restoredState = new RestoredState<>(panes, nextEvaluationTime, nextSlideTime);
+ }
+
+ // ------------------------------------------------------------------------
// Property access (for testing)
// ------------------------------------------------------------------------
@@ -232,4 +316,20 @@ public abstract class AbstractAlignedProcessingTimeWindowOperator<KEY, IN, OUT,
public String toString() {
return "Window (processing time) (length=" + windowSize + ", slide=" + windowSlide + ')';
}
+
+ // ------------------------------------------------------------------------
+ // ------------------------------------------------------------------------
+
+ private static final class RestoredState<IN, KEY, STATE, OUT> {
+
+ final AbstractKeyedTimePanes<IN, KEY, STATE, OUT> panes;
+ final long nextEvaluationTime;
+ final long nextSlideTime;
+
+ RestoredState(AbstractKeyedTimePanes<IN, KEY, STATE, OUT> panes, long nextEvaluationTime, long nextSlideTime) {
+ this.panes = panes;
+ this.nextEvaluationTime = nextEvaluationTime;
+ this.nextSlideTime = nextSlideTime;
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/c24dca50/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractKeyedTimePanes.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractKeyedTimePanes.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractKeyedTimePanes.java
index 07dea06..d1cea20 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractKeyedTimePanes.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractKeyedTimePanes.java
@@ -18,16 +18,29 @@
package org.apache.flink.streaming.runtime.operators.windowing;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
import org.apache.flink.util.Collector;
+import java.io.IOException;
import java.util.ArrayDeque;
+import java.util.Iterator;
-
+/**
+ * Base class for a multiple key/value maps organized in panes.
+ */
public abstract class AbstractKeyedTimePanes<Type, Key, Aggregate, Result> {
+ private static final int BEGIN_OF_STATE_MAGIC_NUMBER = 0x0FF1CE42;
+
+ private static final int BEGIN_OF_PANE_MAGIC_NUMBER = 0xBADF00D5;
+
+ /** The latest time pane */
protected KeyMap<Key, Aggregate> latestPane = new KeyMap<>();
+ /** The previous time panes, ordered by time (early to late) */
protected final ArrayDeque<KeyMap<Key, Aggregate>> previousPanes = new ArrayDeque<>();
// ------------------------------------------------------------------------
@@ -43,6 +56,10 @@ public abstract class AbstractKeyedTimePanes<Type, Key, Aggregate, Result> {
previousPanes.clear();
}
+ public int getNumPanes() {
+ return previousPanes.size() + 1;
+ }
+
public void slidePanes(int panesToKeep) {
if (panesToKeep > 1) {
@@ -74,4 +91,67 @@ public abstract class AbstractKeyedTimePanes<Type, Key, Aggregate, Result> {
// let the maps make a coordinated traversal and evaluate the window function per contained key
KeyMap.traverseMaps(panes, traversal, traversalPass);
}
+
+ // ------------------------------------------------------------------------
+ // Serialization and de-serialization
+ // ------------------------------------------------------------------------
+
+ public void writeToOutput(
+ final DataOutputView output,
+ final TypeSerializer<Key> keySerializer,
+ final TypeSerializer<Aggregate> aggSerializer) throws IOException
+ {
+ output.writeInt(BEGIN_OF_STATE_MAGIC_NUMBER);
+
+ int numPanes = getNumPanes();
+ output.writeInt(numPanes);
+
+ // write from the past
+ Iterator<KeyMap<Key, Aggregate>> previous = previousPanes.iterator();
+ for (int paneNum = 0; paneNum < numPanes; paneNum++) {
+ output.writeInt(BEGIN_OF_PANE_MAGIC_NUMBER);
+ KeyMap<Key, Aggregate> pane = (paneNum == numPanes - 1) ? latestPane : previous.next();
+
+ output.writeInt(pane.size());
+ for (KeyMap.Entry<Key, Aggregate> entry : pane) {
+ keySerializer.serialize(entry.getKey(), output);
+ aggSerializer.serialize(entry.getValue(), output);
+ }
+ }
+ }
+
+ public void readFromInput(
+ final DataInputView input,
+ final TypeSerializer<Key> keySerializer,
+ final TypeSerializer<Aggregate> aggSerializer) throws IOException
+ {
+ validateMagicNumber(BEGIN_OF_STATE_MAGIC_NUMBER, input.readInt());
+ int numPanes = input.readInt();
+
+ // read from the past towards the presence
+ while (numPanes > 0) {
+ validateMagicNumber(BEGIN_OF_PANE_MAGIC_NUMBER, input.readInt());
+ KeyMap<Key, Aggregate> pane = (numPanes == 1) ? latestPane : new KeyMap<Key, Aggregate>();
+
+ final int numElementsInPane = input.readInt();
+ for (int i = numElementsInPane - 1; i >= 0; i--) {
+ Key k = keySerializer.deserialize(input);
+ Aggregate a = aggSerializer.deserialize(input);
+ pane.put(k, a);
+ }
+
+ if (numPanes > 1) {
+ previousPanes.addLast(pane);
+ }
+ numPanes--;
+ }
+ }
+
+ private static void validateMagicNumber(int expected, int found) throws IOException {
+ if (expected != found) {
+ throw new IOException("Corrupt state stream - wrong magic number. " +
+ "Expected '" + Integer.toHexString(expected) +
+ "', found '" + Integer.toHexString(found) + '\'');
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/c24dca50/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingKeyedTimePanes.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingKeyedTimePanes.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingKeyedTimePanes.java
index 55c1be0..c854e6c 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingKeyedTimePanes.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingKeyedTimePanes.java
@@ -35,8 +35,10 @@ public class AccumulatingKeyedTimePanes<Type, Key, Result> extends AbstractKeyed
private final KeyMap.LazyFactory<ArrayList<Type>> listFactory = getListFactory();
private final WindowFunction<Type, Result, Key, Window> function;
-
- private long evaluationPass;
+
+ /**
+ * IMPORTANT: This value needs to start at one, so it is fresher than the value that new entries have (zero) */
+ private long evaluationPass = 1L;
// ------------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/c24dca50/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingProcessingTimeWindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingProcessingTimeWindowOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingProcessingTimeWindowOperator.java
index 3bcffbc..7a7d04c 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingProcessingTimeWindowOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AccumulatingProcessingTimeWindowOperator.java
@@ -19,14 +19,20 @@
package org.apache.flink.streaming.runtime.operators.windowing;
import org.apache.flink.api.common.functions.Function;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
import org.apache.flink.streaming.api.windowing.windows.Window;
+import java.io.IOException;
+import java.util.ArrayList;
+
public class AccumulatingProcessingTimeWindowOperator<KEY, IN, OUT>
- extends AbstractAlignedProcessingTimeWindowOperator<KEY, IN, OUT, WindowFunction<IN, OUT, KEY, TimeWindow>> {
+ extends AbstractAlignedProcessingTimeWindowOperator<KEY, IN, OUT, ArrayList<IN>, WindowFunction<IN, OUT, KEY, TimeWindow>> {
private static final long serialVersionUID = 7305948082830843475L;
@@ -34,10 +40,13 @@ public class AccumulatingProcessingTimeWindowOperator<KEY, IN, OUT>
public AccumulatingProcessingTimeWindowOperator(
WindowFunction<IN, OUT, KEY, TimeWindow> function,
KeySelector<IN, KEY> keySelector,
+ TypeSerializer<KEY> keySerializer,
+ TypeSerializer<IN> valueSerializer,
long windowLength,
long windowSlide)
{
- super(function, keySelector, windowLength, windowSlide);
+ super(function, keySelector, keySerializer,
+ new ArrayListSerializer<IN>(valueSerializer), windowLength, windowSlide);
}
@Override
@@ -47,4 +56,108 @@ public class AccumulatingProcessingTimeWindowOperator<KEY, IN, OUT>
return new AccumulatingKeyedTimePanes<>(keySelector, windowFunction);
}
+
+ // ------------------------------------------------------------------------
+ // Utility Serializer for Lists of Elements
+ // ------------------------------------------------------------------------
+
+ @SuppressWarnings("ForLoopReplaceableByForEach")
+ private static final class ArrayListSerializer<T> extends TypeSerializer<ArrayList<T>> {
+
+ private static final long serialVersionUID = 1119562170939152304L;
+
+ private final TypeSerializer<T> elementSerializer;
+
+ ArrayListSerializer(TypeSerializer<T> elementSerializer) {
+ this.elementSerializer = elementSerializer;
+ }
+
+ @Override
+ public boolean isImmutableType() {
+ return false;
+ }
+
+ @Override
+ public TypeSerializer<ArrayList<T>> duplicate() {
+ TypeSerializer<T> duplicateElement = elementSerializer.duplicate();
+ return duplicateElement == elementSerializer ? this : new ArrayListSerializer<T>(duplicateElement);
+ }
+
+ @Override
+ public ArrayList<T> createInstance() {
+ return new ArrayList<>();
+ }
+
+ @Override
+ public ArrayList<T> copy(ArrayList<T> from) {
+ ArrayList<T> newList = new ArrayList<>(from.size());
+ for (int i = 0; i < from.size(); i++) {
+ newList.add(elementSerializer.copy(from.get(i)));
+ }
+ return newList;
+ }
+
+ @Override
+ public ArrayList<T> copy(ArrayList<T> from, ArrayList<T> reuse) {
+ return copy(from);
+ }
+
+ @Override
+ public int getLength() {
+ return -1; // var length
+ }
+
+ @Override
+ public void serialize(ArrayList<T> list, DataOutputView target) throws IOException {
+ final int size = list.size();
+ target.writeInt(size);
+ for (int i = 0; i < size; i++) {
+ elementSerializer.serialize(list.get(i), target);
+ }
+ }
+
+ @Override
+ public ArrayList<T> deserialize(DataInputView source) throws IOException {
+ final int size = source.readInt();
+ final ArrayList<T> list = new ArrayList<>(size);
+ for (int i = 0; i < size; i++) {
+ list.add(elementSerializer.deserialize(source));
+ }
+ return list;
+ }
+
+ @Override
+ public ArrayList<T> deserialize(ArrayList<T> reuse, DataInputView source) throws IOException {
+ return deserialize(source);
+ }
+
+ @Override
+ public void copy(DataInputView source, DataOutputView target) throws IOException {
+ // copy number of elements
+ final int num = source.readInt();
+ target.writeInt(num);
+ for (int i = 0; i < num; i++) {
+ elementSerializer.copy(source, target);
+ }
+ }
+
+ // --------------------------------------------------------------------
+
+ @Override
+ public boolean equals(Object obj) {
+ return obj == this ||
+ (obj != null && obj.getClass() == getClass() &&
+ elementSerializer.equals(((ArrayListSerializer<?>) obj).elementSerializer));
+ }
+
+ @Override
+ public boolean canEqual(Object obj) {
+ return true;
+ }
+
+ @Override
+ public int hashCode() {
+ return elementSerializer.hashCode();
+ }
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/c24dca50/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingKeyedTimePanes.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingKeyedTimePanes.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingKeyedTimePanes.java
index c17f0b4..d395b2a 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingKeyedTimePanes.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingKeyedTimePanes.java
@@ -29,8 +29,10 @@ public class AggregatingKeyedTimePanes<Type, Key> extends AbstractKeyedTimePanes
private final KeySelector<Type, Key> keySelector;
private final ReduceFunction<Type> reducer;
-
- private long evaluationPass;
+
+ /**
+ * IMPORTANT: This value needs to start at one, so it is fresher than the value that new entries have (zero) */
+ private long evaluationPass = 1L;
// ------------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/c24dca50/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingProcessingTimeWindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingProcessingTimeWindowOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingProcessingTimeWindowOperator.java
index cc38019..0e07cea 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingProcessingTimeWindowOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AggregatingProcessingTimeWindowOperator.java
@@ -20,10 +20,11 @@ package org.apache.flink.streaming.runtime.operators.windowing;
import org.apache.flink.api.common.functions.Function;
import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.java.functions.KeySelector;
public class AggregatingProcessingTimeWindowOperator<KEY, IN>
- extends AbstractAlignedProcessingTimeWindowOperator<KEY, IN, IN, ReduceFunction<IN>> {
+ extends AbstractAlignedProcessingTimeWindowOperator<KEY, IN, IN, IN, ReduceFunction<IN>> {
private static final long serialVersionUID = 7305948082830843475L;
@@ -31,10 +32,12 @@ public class AggregatingProcessingTimeWindowOperator<KEY, IN>
public AggregatingProcessingTimeWindowOperator(
ReduceFunction<IN> function,
KeySelector<IN, KEY> keySelector,
+ TypeSerializer<KEY> keySerializer,
+ TypeSerializer<IN> aggregateSerializer,
long windowLength,
long windowSlide)
{
- super(function, keySelector, windowLength, windowSlide);
+ super(function, keySelector, keySerializer, aggregateSerializer, windowLength, windowSlide);
}
@Override
http://git-wip-us.apache.org/repos/asf/flink/blob/c24dca50/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
index bbfd233..5bf7d8e 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
@@ -188,7 +188,11 @@ public abstract class StreamTask<OUT, Operator extends StreamOperator<OUT>>
boolean disposed = false;
try {
- openAllOperators();
+ // we need to make sure that any triggers scheduled in open() cannot be
+ // executed before all operators are opened
+ synchronized (lock) {
+ openAllOperators();
+ }
// let the task do its work
isRunning = true;
@@ -202,12 +206,13 @@ public abstract class StreamTask<OUT, Operator extends StreamOperator<OUT>>
// make sure no further checkpoint and notification actions happen.
// we make sure that no other thread is currently in the locked scope before
// we close the operators by trying to acquire the checkpoint scope lock
- synchronized (lock) {}
-
- // this is part of the main logic, so if this fails, the task is considered failed
- closeAllOperators();
+ // we also need to make sure that no triggers fire concurrently with the close logic
+ synchronized (lock) {
+ // this is part of the main logic, so if this fails, the task is considered failed
+ closeAllOperators();
+ }
- // make sure all data is flushed
+ // make sure all buffered data is flushed
operatorChain.flushOutputs();
// make an attempt to dispose the operators such that failures in the dispose call
@@ -239,6 +244,14 @@ public abstract class StreamTask<OUT, Operator extends StreamOperator<OUT>>
if (!disposed) {
disposeAllOperators();
}
+
+ try {
+ if (stateBackend != null) {
+ stateBackend.close();
+ }
+ } catch (Throwable t) {
+ LOG.error("Error while closing the state backend", t);
+ }
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/c24dca50/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.java
index aeb5078..dd8dec9 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/AggregationFunctionTest.java
@@ -24,11 +24,13 @@ import java.util.ArrayList;
import java.util.List;
import com.google.common.collect.ImmutableList;
+
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.functions.ReduceFunction;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.api.java.operators.Keys;
+import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.tuple.Tuple3;
import org.apache.flink.api.java.typeutils.TypeExtractor;
@@ -38,6 +40,7 @@ import org.apache.flink.streaming.api.functions.aggregation.SumAggregator;
import org.apache.flink.streaming.api.operators.StreamGroupedReduce;
import org.apache.flink.streaming.util.MockContext;
import org.apache.flink.streaming.util.keys.KeySelectorUtil;
+
import org.junit.Test;
public class AggregationFunctionTest {
@@ -78,9 +81,10 @@ public class AggregationFunctionTest {
ExecutionConfig config = new ExecutionConfig();
- KeySelector<Tuple2<Integer, Integer>, ?> keySelector = KeySelectorUtil.getSelectorForKeys(
+ KeySelector<Tuple2<Integer, Integer>, Tuple> keySelector = KeySelectorUtil.getSelectorForKeys(
new Keys.ExpressionKeys<>(new int[]{0}, typeInfo),
typeInfo, config);
+ TypeInformation<Tuple> keyType = TypeExtractor.getKeySelectorTypes(keySelector, typeInfo);
// aggregations tested
ReduceFunction<Tuple2<Integer, Integer>> sumFunction =
@@ -90,17 +94,20 @@ public class AggregationFunctionTest {
ReduceFunction<Tuple2<Integer, Integer>> maxFunction = new ComparableAggregator<>(
1, typeInfo, AggregationType.MAX, config);
- List<Tuple2<Integer, Integer>> groupedSumList = MockContext.createAndExecute(
+ List<Tuple2<Integer, Integer>> groupedSumList = MockContext.createAndExecuteForKeyedStream(
new StreamGroupedReduce<>(sumFunction, typeInfo.createSerializer(config)),
- getInputList());
+ getInputList(),
+ keySelector, keyType);
- List<Tuple2<Integer, Integer>> groupedMinList = MockContext.createAndExecute(
+ List<Tuple2<Integer, Integer>> groupedMinList = MockContext.createAndExecuteForKeyedStream(
new StreamGroupedReduce<>(minFunction, typeInfo.createSerializer(config)),
- getInputList());
+ getInputList(),
+ keySelector, keyType);
- List<Tuple2<Integer, Integer>> groupedMaxList = MockContext.createAndExecute(
+ List<Tuple2<Integer, Integer>> groupedMaxList = MockContext.createAndExecuteForKeyedStream(
new StreamGroupedReduce<>(maxFunction, typeInfo.createSerializer(config)),
- getInputList());
+ getInputList(),
+ keySelector, keyType);
assertEquals(expectedGroupSumList, groupedSumList);
assertEquals(expectedGroupMinList, groupedMinList);
@@ -143,9 +150,10 @@ public class AggregationFunctionTest {
ExecutionConfig config = new ExecutionConfig();
- KeySelector<MyPojo, ?> keySelector = KeySelectorUtil.getSelectorForKeys(
+ KeySelector<MyPojo, Tuple> keySelector = KeySelectorUtil.getSelectorForKeys(
new Keys.ExpressionKeys<>(new String[]{"f0"}, typeInfo),
typeInfo, config);
+ TypeInformation<Tuple> keyType = TypeExtractor.getKeySelectorTypes(keySelector, typeInfo);
// aggregations tested
ReduceFunction<MyPojo> sumFunction = new SumAggregator<>("f1", typeInfo, config);
@@ -154,15 +162,20 @@ public class AggregationFunctionTest {
ReduceFunction<MyPojo> maxFunction = new ComparableAggregator<>("f1", typeInfo, AggregationType.MAX,
false, config);
- List<MyPojo> groupedSumList = MockContext.createAndExecute(
+ List<MyPojo> groupedSumList = MockContext.createAndExecuteForKeyedStream(
new StreamGroupedReduce<>(sumFunction, typeInfo.createSerializer(config)),
- getInputPojoList());
- List<MyPojo> groupedMinList = MockContext.createAndExecute(
+ getInputPojoList(),
+ keySelector, keyType);
+
+ List<MyPojo> groupedMinList = MockContext.createAndExecuteForKeyedStream(
new StreamGroupedReduce<>(minFunction, typeInfo.createSerializer(config)),
- getInputPojoList());
- List<MyPojo> groupedMaxList = MockContext.createAndExecute(
+ getInputPojoList(),
+ keySelector, keyType);
+
+ List<MyPojo> groupedMaxList = MockContext.createAndExecuteForKeyedStream(
new StreamGroupedReduce<>(maxFunction, typeInfo.createSerializer(config)),
- getInputPojoList());
+ getInputPojoList(),
+ keySelector, keyType);
assertEquals(expectedGroupSumList, groupedSumList);
assertEquals(expectedGroupMinList, groupedMinList);
@@ -200,9 +213,10 @@ public class AggregationFunctionTest {
ExecutionConfig config = new ExecutionConfig();
- KeySelector<Tuple3<Integer, Integer, Integer>, ?> keySelector = KeySelectorUtil.getSelectorForKeys(
+ KeySelector<Tuple3<Integer, Integer, Integer>, Tuple> keySelector = KeySelectorUtil.getSelectorForKeys(
new Keys.ExpressionKeys<>(new int[]{0}, typeInfo),
typeInfo, config);
+ TypeInformation<Tuple> keyType = TypeExtractor.getKeySelectorTypes(keySelector, typeInfo);
// aggregations tested
ReduceFunction<Tuple3<Integer, Integer, Integer>> maxByFunctionFirst =
@@ -214,18 +228,25 @@ public class AggregationFunctionTest {
ReduceFunction<Tuple3<Integer, Integer, Integer>> minByFunctionLast =
new ComparableAggregator<>(1, typeInfo, AggregationType.MINBY, false, config);
- assertEquals(maxByFirstExpected, MockContext.createAndExecute(
+ assertEquals(maxByFirstExpected, MockContext.createAndExecuteForKeyedStream(
new StreamGroupedReduce<>(maxByFunctionFirst, typeInfo.createSerializer(config)),
- getInputByList()));
- assertEquals(maxByLastExpected, MockContext.createAndExecute(
+ getInputByList(),
+ keySelector, keyType));
+
+ assertEquals(maxByLastExpected, MockContext.createAndExecuteForKeyedStream(
new StreamGroupedReduce<>(maxByFunctionLast, typeInfo.createSerializer(config)),
- getInputByList()));
- assertEquals(minByLastExpected, MockContext.createAndExecute(
+ getInputByList(),
+ keySelector, keyType));
+
+ assertEquals(minByLastExpected, MockContext.createAndExecuteForKeyedStream(
new StreamGroupedReduce<>(minByFunctionLast, typeInfo.createSerializer(config)),
- getInputByList()));
- assertEquals(minByFirstExpected, MockContext.createAndExecute(
+ getInputByList(),
+ keySelector, keyType));
+
+ assertEquals(minByFirstExpected, MockContext.createAndExecuteForKeyedStream(
new StreamGroupedReduce<>(minByFunctionFirst, typeInfo.createSerializer(config)),
- getInputByList()));
+ getInputByList(),
+ keySelector, keyType));
}
@Test
@@ -258,9 +279,10 @@ public class AggregationFunctionTest {
ExecutionConfig config = new ExecutionConfig();
- KeySelector<MyPojo3, ?> keySelector = KeySelectorUtil.getSelectorForKeys(
+ KeySelector<MyPojo3, Tuple> keySelector = KeySelectorUtil.getSelectorForKeys(
new Keys.ExpressionKeys<>(new String[]{"f0"}, typeInfo),
typeInfo, config);
+ TypeInformation<Tuple> keyType = TypeExtractor.getKeySelectorTypes(keySelector, typeInfo);
// aggregations tested
ReduceFunction<MyPojo3> maxByFunctionFirst =
@@ -272,18 +294,25 @@ public class AggregationFunctionTest {
ReduceFunction<MyPojo3> minByFunctionLast =
new ComparableAggregator<>("f1", typeInfo, AggregationType.MINBY, false, config);
- assertEquals(maxByFirstExpected, MockContext.createAndExecute(
- new StreamGroupedReduce<>(maxByFunctionFirst, typeInfo.createSerializer(config)),
- getInputByPojoList()));
- assertEquals(maxByLastExpected, MockContext.createAndExecute(
+ assertEquals(maxByFirstExpected, MockContext.createAndExecuteForKeyedStream(
+ new StreamGroupedReduce<>(maxByFunctionFirst, typeInfo.createSerializer(config)),
+ getInputByPojoList(),
+ keySelector, keyType));
+
+ assertEquals(maxByLastExpected, MockContext.createAndExecuteForKeyedStream(
new StreamGroupedReduce<>(maxByFunctionLast, typeInfo.createSerializer(config)),
- getInputByPojoList()));
- assertEquals(minByLastExpected, MockContext.createAndExecute(
+ getInputByPojoList(),
+ keySelector, keyType));
+
+ assertEquals(minByLastExpected, MockContext.createAndExecuteForKeyedStream(
new StreamGroupedReduce<>(minByFunctionLast, typeInfo.createSerializer(config)),
- getInputByPojoList()));
- assertEquals(minByFirstExpected, MockContext.createAndExecute(
+ getInputByPojoList(),
+ keySelector, keyType));
+
+ assertEquals(minByFirstExpected, MockContext.createAndExecuteForKeyedStream(
new StreamGroupedReduce<>(minByFunctionFirst, typeInfo.createSerializer(config)),
- getInputByPojoList()));
+ getInputByPojoList(),
+ keySelector, keyType));
}
// *************************************************************************
http://git-wip-us.apache.org/repos/asf/flink/blob/c24dca50/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/SlotAllocationTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/SlotAllocationTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/SlotAllocationTest.java
index 39a13b3..8038cfb 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/SlotAllocationTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/graph/SlotAllocationTest.java
@@ -25,23 +25,19 @@ import org.apache.flink.api.common.functions.FilterFunction;
import org.apache.flink.runtime.jobgraph.JobVertex;
import org.apache.flink.runtime.jobgraph.JobGraph;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.util.StreamingMultipleProgramsTestBase;
-import org.junit.Test;
-public class SlotAllocationTest extends StreamingMultipleProgramsTestBase{
+import org.junit.Test;
- @SuppressWarnings("serial")
+@SuppressWarnings("serial")
+public class SlotAllocationTest {
+
@Test
public void test() {
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
FilterFunction<Long> dummyFilter = new FilterFunction<Long>() {
-
@Override
- public boolean filter(Long value) throws Exception {
-
- return false;
- }
+ public boolean filter(Long value) { return false; }
};
env.generateSequence(1, 10).filter(dummyFilter).isolateResources().filter(dummyFilter)
@@ -53,11 +49,8 @@ public class SlotAllocationTest extends StreamingMultipleProgramsTestBase{
List<JobVertex> vertices = jobGraph.getVerticesSortedTopologicallyFromSources();
assertEquals(vertices.get(0).getSlotSharingGroup(), vertices.get(2).getSlotSharingGroup());
- assertNotEquals(vertices.get(0).getSlotSharingGroup(), vertices.get(1)
- .getSlotSharingGroup());
- assertNotEquals(vertices.get(2).getSlotSharingGroup(), vertices.get(3)
- .getSlotSharingGroup());
+ assertNotEquals(vertices.get(0).getSlotSharingGroup(), vertices.get(1).getSlotSharingGroup());
+ assertNotEquals(vertices.get(2).getSlotSharingGroup(), vertices.get(3).getSlotSharingGroup());
assertEquals(vertices.get(3).getSlotSharingGroup(), vertices.get(4).getSlotSharingGroup());
-
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/c24dca50/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedFoldTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedFoldTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedFoldTest.java
index 1002b10..f6e7e6b 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedFoldTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedFoldTest.java
@@ -23,17 +23,18 @@ import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.functions.FoldFunction;
import org.apache.flink.api.common.functions.RichFoldFunction;
import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
import org.apache.flink.streaming.util.TestHarnessUtil;
-import org.junit.Assert;
+
import org.junit.Test;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
/**
* Tests for {@link StreamGroupedFold}. These test that:
*
@@ -48,18 +49,12 @@ public class StreamGroupedFoldTest {
private static class MyFolder implements FoldFunction<Integer, String> {
- private static final long serialVersionUID = 1L;
-
@Override
public String fold(String accumulator, Integer value) throws Exception {
return accumulator + value.toString();
}
-
}
- private TypeInformation<Integer> inType = TypeExtractor.getForClass(Integer.class);
- private TypeInformation<String> outType = TypeExtractor.getForClass(String.class);
-
@Test
public void testGroupedFold() throws Exception {
@@ -72,9 +67,10 @@ public class StreamGroupedFoldTest {
};
StreamGroupedFold<Integer, String, String> operator = new StreamGroupedFold<>(new MyFolder(), "100");
- operator.setOutputType(outType, new ExecutionConfig());
+ operator.setOutputType(BasicTypeInfo.STRING_TYPE_INFO, new ExecutionConfig());
OneInputStreamOperatorTestHarness<Integer, String> testHarness = new OneInputStreamOperatorTestHarness<>(operator);
+ testHarness.configureForKeyedStream(keySelector, BasicTypeInfo.STRING_TYPE_INFO);
long initialTime = 0L;
ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
@@ -112,7 +108,9 @@ public class StreamGroupedFoldTest {
operator.setOutputType(BasicTypeInfo.STRING_TYPE_INFO, new ExecutionConfig());
OneInputStreamOperatorTestHarness<Integer, String> testHarness = new OneInputStreamOperatorTestHarness<>(operator);
-
+ testHarness.configureForKeyedStream(keySelector, BasicTypeInfo.INT_TYPE_INFO);
+
+
long initialTime = 0L;
testHarness.open();
@@ -122,8 +120,8 @@ public class StreamGroupedFoldTest {
testHarness.close();
- Assert.assertTrue("RichFunction methods where not called.", TestOpenCloseFoldFunction.closeCalled);
- Assert.assertTrue("Output contains no elements.", testHarness.getOutput().size() > 0);
+ assertTrue("RichFunction methods where not called.", TestOpenCloseFoldFunction.closeCalled);
+ assertTrue("Output contains no elements.", testHarness.getOutput().size() > 0);
}
// This must only be used in one test, otherwise the static fields will be changed
@@ -138,7 +136,7 @@ public class StreamGroupedFoldTest {
public void open(Configuration parameters) throws Exception {
super.open(parameters);
if (closeCalled) {
- Assert.fail("Close called before open.");
+ fail("Close called before open.");
}
openCalled = true;
}
@@ -147,7 +145,7 @@ public class StreamGroupedFoldTest {
public void close() throws Exception {
super.close();
if (!openCalled) {
- Assert.fail("Open was not called before close.");
+ fail("Open was not called before close.");
}
closeCalled = true;
}
@@ -155,7 +153,7 @@ public class StreamGroupedFoldTest {
@Override
public String fold(String acc, Integer in) throws Exception {
if (!openCalled) {
- Assert.fail("Open was not called before run.");
+ fail("Open was not called before run.");
}
return acc + in;
}
http://git-wip-us.apache.org/repos/asf/flink/blob/c24dca50/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedReduceTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedReduceTest.java b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedReduceTest.java
index b5d2bd6..6cb46c9 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedReduceTest.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/test/java/org/apache/flink/streaming/api/operators/StreamGroupedReduceTest.java
@@ -53,6 +53,7 @@ public class StreamGroupedReduceTest {
StreamGroupedReduce<Integer> operator = new StreamGroupedReduce<>(new MyReducer(), IntSerializer.INSTANCE);
OneInputStreamOperatorTestHarness<Integer, Integer> testHarness = new OneInputStreamOperatorTestHarness<>(operator);
+ testHarness.configureForKeyedStream(keySelector, BasicTypeInfo.INT_TYPE_INFO);
long initialTime = 0L;
ConcurrentLinkedQueue<Object> expectedOutput = new ConcurrentLinkedQueue<>();
@@ -84,6 +85,7 @@ public class StreamGroupedReduceTest {
StreamGroupedReduce<Integer> operator =
new StreamGroupedReduce<>(new TestOpenCloseReduceFunction(), IntSerializer.INSTANCE);
OneInputStreamOperatorTestHarness<Integer, Integer> testHarness = new OneInputStreamOperatorTestHarness<>(operator);
+ testHarness.configureForKeyedStream(keySelector, BasicTypeInfo.INT_TYPE_INFO);
long initialTime = 0L;
[10/24] flink git commit: [FLINK-2808] [streaming] Refactor and
extend state backend abstraction
Posted by se...@apache.org.
[FLINK-2808] [streaming] Refactor and extend state backend abstraction
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/479bec0b
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/479bec0b
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/479bec0b
Branch: refs/heads/master
Commit: 479bec0b2b44315196c1f1cddeb114c79d1717db
Parents: 5ac2872
Author: Stephan Ewen <se...@apache.org>
Authored: Mon Oct 5 15:57:04 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 16 15:26:10 2015 +0200
----------------------------------------------------------------------
.../flink/storm/wrappers/BoltWrapper.java | 12 +-
.../flink/storm/wrappers/SpoutWrapper.java | 2 +-
.../storm/wrappers/WrapperSetupHelper.java | 2 +-
.../storm/api/FlinkTopologyBuilderTest.java | 6 +-
.../flink/storm/wrappers/BoltWrapperTest.java | 161 ++++---
.../flink/storm/wrappers/SpoutWrapperTest.java | 11 +-
.../storm/wrappers/WrapperSetupHelperTest.java | 12 +-
.../common/functions/AbstractRichFunction.java | 5 +-
.../functions/IterationRuntimeContext.java | 2 +-
.../api/common/functions/RichFunction.java | 28 +-
.../api/common/functions/RuntimeContext.java | 163 ++++---
.../util/AbstractRuntimeUDFContext.java | 17 +-
.../common/functions/util/ListCollector.java | 5 +
.../flink/api/common/state/OperatorState.java | 23 +-
.../api/common/state/StateCheckpointer.java | 73 ---
.../flink/configuration/ConfigConstants.java | 5 -
.../memory/InputViewDataInputStreamWrapper.java | 5 +-
.../apache/flink/util/InstantiationUtil.java | 7 +-
.../org/apache/flink/util/SerializedValue.java | 8 +
.../flink/core/testutils/CommonTestUtils.java | 21 +-
flink-dist/src/main/resources/flink-conf.yaml | 15 +-
.../flink/runtime/state/FileStateHandle.java | 31 --
.../flink/runtime/state/LocalStateHandle.java | 15 +-
.../runtime/state/StateHandleProvider.java | 39 --
.../apache/flink/runtime/taskmanager/Task.java | 2 -
flink-staging/flink-fs-tests/pom.xml | 19 +
.../flink/hdfstests/FileStateBackendTest.java | 308 +++++++++++++
.../flink/hdfstests/FileStateHandleTest.java | 126 ------
.../kafka/testutils/MockRuntimeContext.java | 37 +-
.../BroadcastOutputSelectorWrapper.java | 12 +-
.../selector/DirectedOutputSelectorWrapper.java | 33 +-
.../selector/OutputSelectorWrapper.java | 2 +-
.../streaming/api/datastream/DataStream.java | 12 +-
.../api/datastream/DataStreamSink.java | 4 +-
.../streaming/api/datastream/KeyedStream.java | 38 +-
.../datastream/SingleOutputStreamOperator.java | 7 +-
.../environment/StreamExecutionEnvironment.java | 64 +--
.../api/functions/sink/FileSinkFunction.java | 10 +-
.../api/functions/sink/PrintSinkFunction.java | 2 +-
.../functions/source/FileSourceFunction.java | 5 +-
.../source/StatefulSequenceSource.java | 36 +-
.../flink/streaming/api/graph/StreamConfig.java | 149 +++----
.../flink/streaming/api/graph/StreamGraph.java | 21 +-
.../api/graph/StreamGraphGenerator.java | 28 +-
.../flink/streaming/api/graph/StreamNode.java | 9 +
.../api/graph/StreamingJobGraphGenerator.java | 19 +-
.../api/operators/AbstractStreamOperator.java | 296 ++++++++++--
.../operators/AbstractUdfStreamOperator.java | 163 ++++---
.../api/operators/ChainingStrategy.java | 47 ++
.../api/operators/OneInputStreamOperator.java | 4 +-
.../flink/streaming/api/operators/Output.java | 2 +-
.../api/operators/StatefulStreamOperator.java | 40 --
.../streaming/api/operators/StreamFlatMap.java | 5 +-
.../api/operators/StreamGroupedFold.java | 47 +-
.../api/operators/StreamGroupedReduce.java | 54 +--
.../streaming/api/operators/StreamOperator.java | 92 ++--
.../streaming/api/operators/StreamProject.java | 5 +-
.../streaming/api/operators/StreamSource.java | 3 +-
.../api/operators/StreamingRuntimeContext.java | 162 +++++++
.../api/operators/co/CoStreamFlatMap.java | 5 +-
.../api/state/AbstractHeapKvState.java | 145 ++++++
.../streaming/api/state/BasicCheckpointer.java | 37 --
.../streaming/api/state/EagerStateStore.java | 104 -----
.../streaming/api/state/KVMapCheckpointer.java | 82 ----
.../flink/streaming/api/state/KvState.java | 69 +++
.../streaming/api/state/KvStateSnapshot.java | 69 +++
.../api/state/OperatorStateHandle.java | 54 ---
.../api/state/PartitionedStateStore.java | 55 ---
.../state/PartitionedStreamOperatorState.java | 182 --------
.../flink/streaming/api/state/StateBackend.java | 135 ++++++
.../api/state/StateBackendFactory.java | 40 ++
.../api/state/StreamOperatorState.java | 132 ------
.../streaming/api/state/StreamStateHandle.java | 28 ++
.../streaming/api/state/WrapperStateHandle.java | 61 ---
.../api/state/filesystem/AbstractFileState.java | 83 ++++
.../filesystem/FileSerializableStateHandle.java | 53 +++
.../state/filesystem/FileStreamStateHandle.java | 46 ++
.../api/state/filesystem/FsHeapKvState.java | 88 ++++
.../state/filesystem/FsHeapKvStateSnapshot.java | 95 ++++
.../api/state/filesystem/FsStateBackend.java | 409 +++++++++++++++++
.../state/filesystem/FsStateBackendFactory.java | 56 +++
.../api/state/memory/ByteStreamStateHandle.java | 52 +++
.../api/state/memory/MemHeapKvState.java | 52 +++
.../state/memory/MemoryHeapKvStateSnapshot.java | 102 +++++
.../api/state/memory/MemoryStateBackend.java | 206 +++++++++
.../api/state/memory/SerializedStateHandle.java | 49 ++
.../CoFeedbackTransformation.java | 4 +-
.../transformations/FeedbackTransformation.java | 4 +-
.../transformations/OneInputTransformation.java | 18 +-
.../PartitionTransformation.java | 6 +-
.../transformations/SelectTransformation.java | 9 +-
.../api/transformations/SinkTransformation.java | 15 +-
.../transformations/SourceTransformation.java | 4 +-
.../transformations/SplitTransformation.java | 4 +-
.../transformations/StreamTransformation.java | 5 +-
.../transformations/TwoInputTransformation.java | 4 +-
.../transformations/UnionTransformation.java | 4 +-
.../streaming/runtime/io/CollectorWrapper.java | 18 +-
.../runtime/io/StreamInputProcessor.java | 9 +-
.../operators/BucketStreamSortOperator.java | 18 +-
.../operators/ExtractTimestampsOperator.java | 14 +-
...ractAlignedProcessingTimeWindowOperator.java | 9 +-
.../windowing/NonKeyedWindowOperator.java | 28 +-
.../operators/windowing/WindowOperator.java | 29 +-
.../ExceptionInChainedOperatorException.java | 45 ++
.../runtime/tasks/OneInputStreamTask.java | 14 +-
.../streaming/runtime/tasks/OperatorChain.java | 308 +++++++++++++
.../streaming/runtime/tasks/OutputHandler.java | 336 --------------
.../runtime/tasks/SourceStreamTask.java | 12 +-
.../runtime/tasks/StreamIterationHead.java | 8 +-
.../runtime/tasks/StreamIterationTail.java | 6 +-
.../streaming/runtime/tasks/StreamTask.java | 447 ++++++++++---------
.../runtime/tasks/StreamTaskState.java | 108 +++++
.../runtime/tasks/StreamTaskStateList.java | 60 +++
.../runtime/tasks/StreamingRuntimeContext.java | 204 ---------
.../runtime/tasks/TwoInputStreamTask.java | 15 +-
.../streaming/api/AggregationFunctionTest.java | 31 +-
.../flink/streaming/api/DataStreamTest.java | 5 +-
.../api/functions/PrintSinkFunctionTest.java | 2 +-
.../api/graph/StreamGraphGeneratorTest.java | 15 +-
.../api/operators/StreamGroupedFoldTest.java | 28 +-
.../api/operators/StreamGroupedReduceTest.java | 17 +-
.../api/state/FileStateBackendTest.java | 419 +++++++++++++++++
.../api/state/MemoryStateBackendTest.java | 278 ++++++++++++
.../streaming/api/state/StateHandleTest.java | 135 ------
.../api/state/StatefulOperatorTest.java | 377 ----------------
...AlignedProcessingTimeWindowOperatorTest.java | 209 +++++----
...AlignedProcessingTimeWindowOperatorTest.java | 201 +++++----
.../runtime/tasks/StreamTaskTestHarness.java | 13 +-
.../runtime/tasks/StreamTaskTimerITCase.java | 17 +-
.../streaming/timestamp/TimestampITCase.java | 4 +-
.../flink/streaming/util/MockContext.java | 74 ++-
.../util/OneInputStreamOperatorTestHarness.java | 71 +--
.../streaming/util/SourceFunctionUtil.java | 19 +-
.../util/TwoInputStreamOperatorTestHarness.java | 69 +--
.../flink/streaming/api/scala/DataStream.scala | 88 +---
.../flink/streaming/api/scala/KeyedStream.scala | 106 ++++-
.../api/scala/StreamExecutionEnvironment.scala | 39 +-
.../api/scala/function/StatefulFunction.scala | 16 +-
.../streaming/api/scala/DataStreamTest.scala | 77 ++--
.../streaming/api/scala/StateTestPrograms.scala | 23 +-
.../CoStreamCheckpointingITCase.java | 73 +--
.../PartitionedStateCheckpointingITCase.java | 52 +--
.../checkpointing/StateCheckpoinedITCase.java | 21 +-
.../StreamCheckpointNotifierITCase.java | 61 ++-
.../StreamCheckpointingITCase.java | 120 ++---
.../UdfStreamOperatorCheckpointingITCase.java | 50 ++-
.../test/classloading/ClassLoaderITCase.java | 4 +-
.../ProcessFailureStreamingRecoveryITCase.java | 38 +-
149 files changed, 5747 insertions(+), 3790 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltWrapper.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltWrapper.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltWrapper.java
index b16fc09..f0913e8 100644
--- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltWrapper.java
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/BoltWrapper.java
@@ -29,7 +29,6 @@ import org.apache.flink.api.common.ExecutionConfig.GlobalJobParameters;
import org.apache.flink.api.java.tuple.Tuple0;
import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.api.java.tuple.Tuple25;
-import org.apache.flink.configuration.Configuration;
import org.apache.flink.storm.util.SplitStreamType;
import org.apache.flink.storm.util.StormConfig;
import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
@@ -62,11 +61,12 @@ public class BoltWrapper<IN, OUT> extends AbstractStreamOperator<OUT> implements
private final Fields inputSchema;
/** The original Storm topology. */
protected StormTopology stormTopology;
+
/**
* We have to use this because Operators must output
* {@link org.apache.flink.streaming.runtime.streamrecord.StreamRecord}.
*/
- private TimestampedCollector<OUT> flinkCollector;
+ private transient TimestampedCollector<OUT> flinkCollector;
/**
* Instantiates a new {@link BoltWrapper} that wraps the given Storm {@link IRichBolt bolt} such that it can be
@@ -206,8 +206,8 @@ public class BoltWrapper<IN, OUT> extends AbstractStreamOperator<OUT> implements
}
@Override
- public void open(final Configuration parameters) throws Exception {
- super.open(parameters);
+ public void open() throws Exception {
+ super.open();
this.flinkCollector = new TimestampedCollector<OUT>(output);
OutputCollector stormCollector = null;
@@ -217,7 +217,7 @@ public class BoltWrapper<IN, OUT> extends AbstractStreamOperator<OUT> implements
this.numberOfAttributes, flinkCollector));
}
- GlobalJobParameters config = super.executionConfig.getGlobalJobParameters();
+ GlobalJobParameters config = getExecutionConfig().getGlobalJobParameters();
StormConfig stormConfig = new StormConfig();
if (config != null) {
@@ -229,7 +229,7 @@ public class BoltWrapper<IN, OUT> extends AbstractStreamOperator<OUT> implements
}
final TopologyContext topologyContext = WrapperSetupHelper.createTopologyContext(
- super.runtimeContext, this.bolt, this.stormTopology, stormConfig);
+ getRuntimeContext(), this.bolt, this.stormTopology, stormConfig);
this.bolt.prepare(stormConfig, topologyContext, stormCollector);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SpoutWrapper.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SpoutWrapper.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SpoutWrapper.java
index 914a19d..e78dd5c 100644
--- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SpoutWrapper.java
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/SpoutWrapper.java
@@ -31,7 +31,7 @@ import org.apache.flink.api.java.tuple.Tuple25;
import org.apache.flink.storm.util.FiniteSpout;
import org.apache.flink.storm.util.StormConfig;
import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
-import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
import com.google.common.collect.Sets;
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/WrapperSetupHelper.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/WrapperSetupHelper.java b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/WrapperSetupHelper.java
index d529b6a..5f1f142 100644
--- a/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/WrapperSetupHelper.java
+++ b/flink-contrib/flink-storm/src/main/java/org/apache/flink/storm/wrappers/WrapperSetupHelper.java
@@ -29,7 +29,7 @@ import backtype.storm.topology.IRichBolt;
import backtype.storm.topology.IRichSpout;
import backtype.storm.tuple.Fields;
-import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
import clojure.lang.Atom;
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyBuilderTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyBuilderTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyBuilderTest.java
index e6fb8e5..906d081 100644
--- a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyBuilderTest.java
+++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/api/FlinkTopologyBuilderTest.java
@@ -16,10 +16,12 @@
*/
package org.apache.flink.storm.api;
-import org.apache.flink.storm.api.FlinkTopologyBuilder;
+
import org.apache.flink.storm.util.TestDummyBolt;
import org.apache.flink.storm.util.TestDummySpout;
import org.apache.flink.storm.util.TestSink;
+
+import org.junit.Ignore;
import org.junit.Test;
import backtype.storm.tuple.Fields;
@@ -52,6 +54,7 @@ public class FlinkTopologyBuilderTest {
}
@Test
+ @Ignore
public void testFieldsGroupingOnMultipleSpoutOutputStreams() {
FlinkTopologyBuilder flinkBuilder = new FlinkTopologyBuilder();
@@ -63,6 +66,7 @@ public class FlinkTopologyBuilderTest {
}
@Test
+ @Ignore
public void testFieldsGroupingOnMultipleBoltOutputStreams() {
FlinkTopologyBuilder flinkBuilder = new FlinkTopologyBuilder();
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltWrapperTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltWrapperTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltWrapperTest.java
index e33fdb9..c1485c8 100644
--- a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltWrapperTest.java
+++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/BoltWrapperTest.java
@@ -29,18 +29,18 @@ import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.api.java.tuple.Tuple1;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.UnmodifiableConfiguration;
+import org.apache.flink.runtime.execution.Environment;
import org.apache.flink.storm.util.AbstractTest;
import org.apache.flink.storm.util.SplitStreamType;
import org.apache.flink.storm.util.StormConfig;
import org.apache.flink.storm.util.TestDummyBolt;
-import org.apache.flink.storm.wrappers.BoltWrapper;
-import org.apache.flink.storm.wrappers.SetupOutputFieldsDeclarer;
-import org.apache.flink.storm.wrappers.StormTuple;
-import org.apache.flink.storm.wrappers.WrapperSetupHelper;
+import org.apache.flink.streaming.api.graph.StreamConfig;
import org.apache.flink.streaming.api.operators.Output;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer;
-import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
@@ -139,7 +139,6 @@ public class BoltWrapperTest extends AbstractTest {
final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class);
when(taskContext.getExecutionConfig()).thenReturn(mock(ExecutionConfig.class));
- when(taskContext.getTaskStubParameters()).thenReturn(new Configuration());
when(taskContext.getTaskName()).thenReturn("name");
final IRichBolt bolt = mock(IRichBolt.class);
@@ -149,8 +148,8 @@ public class BoltWrapperTest extends AbstractTest {
PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer);
final BoltWrapper wrapper = new BoltWrapper(bolt, (Fields) null);
- wrapper.setup(mock(Output.class), taskContext);
- wrapper.open(null);
+ wrapper.setup(createMockStreamTask(), new StreamConfig(new Configuration()), mock(Output.class));
+ wrapper.open();
wrapper.processElement(record);
if (numberOfAttributes == -1) {
@@ -169,11 +168,6 @@ public class BoltWrapperTest extends AbstractTest {
final StreamRecord record = mock(StreamRecord.class);
when(record.getValue()).thenReturn(2).thenReturn(3);
- final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class);
- when(taskContext.getExecutionConfig()).thenReturn(mock(ExecutionConfig.class));
- when(taskContext.getTaskStubParameters()).thenReturn(new Configuration());
- when(taskContext.getTaskName()).thenReturn("name");
-
final Output output = mock(Output.class);
final TestBolt bolt = new TestBolt();
@@ -186,8 +180,8 @@ public class BoltWrapperTest extends AbstractTest {
}
final BoltWrapper wrapper = new BoltWrapper(bolt, (Fields) null, raw);
- wrapper.setup(output, taskContext);
- wrapper.open(null);
+ wrapper.setup(createMockStreamTask(), new StreamConfig(new Configuration()), output);
+ wrapper.open();
final SplitStreamType splitRecord = new SplitStreamType<Integer>();
if (rawOutType1) {
@@ -214,86 +208,70 @@ public class BoltWrapperTest extends AbstractTest {
@SuppressWarnings("unchecked")
@Test
public void testOpen() throws Exception {
- final StormConfig stormConfig = new StormConfig();
- final Configuration flinkConfig = new Configuration();
-
- final ExecutionConfig taskConfig = mock(ExecutionConfig.class);
- when(taskConfig.getGlobalJobParameters()).thenReturn(null).thenReturn(stormConfig)
- .thenReturn(flinkConfig);
-
- final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class);
- when(taskContext.getExecutionConfig()).thenReturn(taskConfig);
- when(taskContext.getTaskStubParameters()).thenReturn(new Configuration());
- when(taskContext.getTaskName()).thenReturn("name");
-
+
+ // utility mocks
final SetupOutputFieldsDeclarer declarer = new SetupOutputFieldsDeclarer();
declarer.declare(new Fields("dummy"));
PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer);
+
+ // (1) open with no configuration
+ {
+ ExecutionConfig execConfig = mock(ExecutionConfig.class);
+ when(execConfig.getGlobalJobParameters()).thenReturn(null);
+
+ final IRichBolt bolt = mock(IRichBolt.class);
+ BoltWrapper<Object, Object> wrapper = new BoltWrapper<Object, Object>(bolt);
+ wrapper.setup(createMockStreamTask(execConfig), new StreamConfig(new Configuration()), mock(Output.class));
+
+ wrapper.open();
+ verify(bolt).prepare(any(Map.class), any(TopologyContext.class), any(OutputCollector.class));
+ }
- final IRichBolt bolt = mock(IRichBolt.class);
+ // (2) open with a storm specific configuration
+ {
+ final StormConfig stormConfig = new StormConfig();
- BoltWrapper<Object, Object> wrapper = new BoltWrapper<Object, Object>(bolt);
- wrapper.setup(mock(Output.class), taskContext);
+ ExecutionConfig execConfig = mock(ExecutionConfig.class);
+ when(execConfig.getGlobalJobParameters()).thenReturn(stormConfig);
+
+ final IRichBolt bolt = mock(IRichBolt.class);
+ BoltWrapper<Object, Object> wrapper = new BoltWrapper<Object, Object>(bolt);
+ wrapper.setup(createMockStreamTask(execConfig), new StreamConfig(new Configuration()), mock(Output.class));
+
+ wrapper.open();
+ verify(bolt).prepare(same(stormConfig), any(TopologyContext.class), any(OutputCollector.class));
+ }
- // test without configuration
- wrapper.open(null);
- verify(bolt).prepare(any(Map.class), any(TopologyContext.class), any(OutputCollector.class));
+ // (3) open with a flink config
+ {
+ final Configuration cfg = new Configuration();
+ cfg.setString("foo", "bar");
+ cfg.setInteger("the end (the int)", Integer.MAX_VALUE);
- // test with StormConfig
- wrapper.open(null);
- verify(bolt).prepare(same(stormConfig), any(TopologyContext.class),
- any(OutputCollector.class));
+ ExecutionConfig execConfig = mock(ExecutionConfig.class);
+ when(execConfig.getGlobalJobParameters()).thenReturn(new UnmodifiableConfiguration(cfg));
- // test with Configuration
- final TestDummyBolt testBolt = new TestDummyBolt();
- wrapper = new BoltWrapper<Object, Object>(testBolt);
- wrapper.setup(mock(Output.class), taskContext);
+ TestDummyBolt testBolt = new TestDummyBolt();
+ BoltWrapper<Object, Object> wrapper = new BoltWrapper<Object, Object>(testBolt);
+ wrapper.setup(createMockStreamTask(execConfig), new StreamConfig(new Configuration()), mock(Output.class));
- wrapper.open(null);
- for (Entry<String, String> entry : flinkConfig.toMap().entrySet()) {
- Assert.assertEquals(entry.getValue(), testBolt.config.get(entry.getKey()));
+ wrapper.open();
+ for (Entry<String, String> entry : cfg.toMap().entrySet()) {
+ Assert.assertEquals(entry.getValue(), testBolt.config.get(entry.getKey()));
+ }
}
}
@SuppressWarnings("unchecked")
@Test
public void testOpenSink() throws Exception {
- final StormConfig stormConfig = new StormConfig();
- final Configuration flinkConfig = new Configuration();
-
- final ExecutionConfig taskConfig = mock(ExecutionConfig.class);
- when(taskConfig.getGlobalJobParameters()).thenReturn(null).thenReturn(stormConfig)
- .thenReturn(flinkConfig);
-
- final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class);
- when(taskContext.getExecutionConfig()).thenReturn(taskConfig);
- when(taskContext.getTaskStubParameters()).thenReturn(new Configuration());
- when(taskContext.getTaskName()).thenReturn("name");
-
final IRichBolt bolt = mock(IRichBolt.class);
-
BoltWrapper<Object, Object> wrapper = new BoltWrapper<Object, Object>(bolt);
- wrapper.setup(mock(Output.class), taskContext);
-
- // test without configuration
- wrapper.open(null);
- verify(bolt).prepare(any(Map.class), any(TopologyContext.class),
- isNull(OutputCollector.class));
-
- // test with StormConfig
- wrapper.open(null);
- verify(bolt).prepare(same(stormConfig), any(TopologyContext.class),
- isNull(OutputCollector.class));
-
- // test with Configuration
- final TestDummyBolt testBolt = new TestDummyBolt();
- wrapper = new BoltWrapper<Object, Object>(testBolt);
- wrapper.setup(mock(Output.class), taskContext);
-
- wrapper.open(null);
- for (Entry<String, String> entry : flinkConfig.toMap().entrySet()) {
- Assert.assertEquals(entry.getValue(), testBolt.config.get(entry.getKey()));
- }
+
+ wrapper.setup(createMockStreamTask(), new StreamConfig(new Configuration()), mock(Output.class));
+ wrapper.open();
+
+ verify(bolt).prepare(any(Map.class), any(TopologyContext.class), isNull(OutputCollector.class));
}
@SuppressWarnings("unchecked")
@@ -306,9 +284,8 @@ public class BoltWrapperTest extends AbstractTest {
PowerMockito.whenNew(SetupOutputFieldsDeclarer.class).withNoArguments().thenReturn(declarer);
final BoltWrapper<Object, Object> wrapper = new BoltWrapper<Object, Object>(bolt);
-
- final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class);
- wrapper.setup(mock(Output.class), taskContext);
+
+ wrapper.setup(createMockStreamTask(), new StreamConfig(new Configuration()), mock(Output.class));
wrapper.close();
wrapper.dispose();
@@ -351,5 +328,25 @@ public class BoltWrapperTest extends AbstractTest {
}
}
-
+ public static StreamTask<?, ?> createMockStreamTask() {
+ return createMockStreamTask(new ExecutionConfig());
+ }
+
+ public static StreamTask<?, ?> createMockStreamTask(ExecutionConfig execConfig) {
+ Environment env = mock(Environment.class);
+ when(env.getTaskName()).thenReturn("Mock Task");
+ when(env.getTaskNameWithSubtasks()).thenReturn("Mock Task (1/1)");
+ when(env.getIndexInSubtaskGroup()).thenReturn(0);
+ when(env.getNumberOfSubtasks()).thenReturn(1);
+ when(env.getUserClassLoader()).thenReturn(BoltWrapperTest.class.getClassLoader());
+
+ StreamTask<?, ?> mockTask = mock(StreamTask.class);
+ when(mockTask.getName()).thenReturn("Mock Task (1/1)");
+ when(mockTask.getCheckpointLock()).thenReturn(new Object());
+ when(mockTask.getConfiguration()).thenReturn(new StreamConfig(new Configuration()));
+ when(mockTask.getEnvironment()).thenReturn(env);
+ when(mockTask.getExecutionConfig()).thenReturn(execConfig);
+
+ return mockTask;
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SpoutWrapperTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SpoutWrapperTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SpoutWrapperTest.java
index 227d736..b81b775 100644
--- a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SpoutWrapperTest.java
+++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/SpoutWrapperTest.java
@@ -30,11 +30,8 @@ import org.apache.flink.storm.util.FiniteSpout;
import org.apache.flink.storm.util.FiniteTestSpout;
import org.apache.flink.storm.util.StormConfig;
import org.apache.flink.storm.util.TestDummySpout;
-import org.apache.flink.storm.wrappers.SetupOutputFieldsDeclarer;
-import org.apache.flink.storm.wrappers.SpoutWrapper;
-import org.apache.flink.storm.wrappers.WrapperSetupHelper;
import org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext;
-import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
@@ -72,7 +69,6 @@ public class SpoutWrapperTest extends AbstractTest {
final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class);
when(taskContext.getExecutionConfig()).thenReturn(taskConfig);
- when(taskContext.getTaskStubParameters()).thenReturn(new Configuration());
when(taskContext.getTaskName()).thenReturn("name");
final IRichSpout spout = mock(IRichSpout.class);
@@ -112,7 +108,6 @@ public class SpoutWrapperTest extends AbstractTest {
final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class);
when(taskContext.getExecutionConfig()).thenReturn(mock(ExecutionConfig.class));
- when(taskContext.getTaskStubParameters()).thenReturn(new Configuration());
when(taskContext.getTaskName()).thenReturn("name");
final IRichSpout spout = mock(IRichSpout.class);
@@ -136,7 +131,6 @@ public class SpoutWrapperTest extends AbstractTest {
final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class);
when(taskContext.getExecutionConfig()).thenReturn(mock(ExecutionConfig.class));
- when(taskContext.getTaskStubParameters()).thenReturn(new Configuration());
when(taskContext.getTaskName()).thenReturn("name");
final FiniteTestSpout spout = new FiniteTestSpout(numberOfCalls);
@@ -158,7 +152,6 @@ public class SpoutWrapperTest extends AbstractTest {
final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class);
when(taskContext.getExecutionConfig()).thenReturn(mock(ExecutionConfig.class));
- when(taskContext.getTaskStubParameters()).thenReturn(new Configuration());
when(taskContext.getTaskName()).thenReturn("name");
final SpoutWrapper<?> wrapper = new SpoutWrapper<Object>(stormSpout);
@@ -176,7 +169,6 @@ public class SpoutWrapperTest extends AbstractTest {
final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class);
when(taskContext.getExecutionConfig()).thenReturn(mock(ExecutionConfig.class));
- when(taskContext.getTaskStubParameters()).thenReturn(new Configuration());
when(taskContext.getTaskName()).thenReturn("name");
final SpoutWrapper<?> wrapper = new SpoutWrapper<Object>(stormSpout);
@@ -192,7 +184,6 @@ public class SpoutWrapperTest extends AbstractTest {
final StreamingRuntimeContext taskContext = mock(StreamingRuntimeContext.class);
when(taskContext.getExecutionConfig()).thenReturn(mock(ExecutionConfig.class));
- when(taskContext.getTaskStubParameters()).thenReturn(new Configuration());
when(taskContext.getTaskName()).thenReturn("name");
final IRichSpout spout = new FiniteTestSpout(numberOfCalls);
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/WrapperSetupHelperTest.java
----------------------------------------------------------------------
diff --git a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/WrapperSetupHelperTest.java b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/WrapperSetupHelperTest.java
index c3b0300..20e480d 100644
--- a/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/WrapperSetupHelperTest.java
+++ b/flink-contrib/flink-storm/src/test/java/org/apache/flink/storm/wrappers/WrapperSetupHelperTest.java
@@ -39,9 +39,7 @@ import org.apache.flink.storm.util.AbstractTest;
import org.apache.flink.storm.util.TestDummyBolt;
import org.apache.flink.storm.util.TestDummySpout;
import org.apache.flink.storm.util.TestSink;
-import org.apache.flink.storm.wrappers.SetupOutputFieldsDeclarer;
-import org.apache.flink.storm.wrappers.WrapperSetupHelper;
-import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext;
+import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
@@ -181,9 +179,9 @@ public class WrapperSetupHelperTest extends AbstractTest {
builder.setBolt("bolt1", (IRichBolt) operators.get("bolt1"), dops.get("bolt1")).shuffleGrouping("spout1");
builder.setBolt("bolt2", (IRichBolt) operators.get("bolt2"), dops.get("bolt2")).allGrouping("spout2");
builder.setBolt("sink", (IRichBolt) operators.get("sink"), dops.get("sink"))
- .fieldsGrouping("bolt1", TestDummyBolt.groupingStreamId, new Fields("id"))
+ .shuffleGrouping("bolt1", TestDummyBolt.groupingStreamId)
.shuffleGrouping("bolt1", TestDummyBolt.shuffleStreamId)
- .fieldsGrouping("bolt2", TestDummyBolt.groupingStreamId, new Fields("id"))
+ .shuffleGrouping("bolt2", TestDummyBolt.groupingStreamId)
.shuffleGrouping("bolt2", TestDummyBolt.shuffleStreamId);
int counter = 0;
@@ -207,9 +205,9 @@ public class WrapperSetupHelperTest extends AbstractTest {
flinkBuilder.setBolt("bolt1", (IRichBolt) operators.get("bolt1"), dops.get("bolt1")).shuffleGrouping("spout1");
flinkBuilder.setBolt("bolt2", (IRichBolt) operators.get("bolt2"), dops.get("bolt2")).allGrouping("spout2");
flinkBuilder.setBolt("sink", (IRichBolt) operators.get("sink"), dops.get("sink"))
- .fieldsGrouping("bolt1", TestDummyBolt.groupingStreamId, new Fields("id"))
+ .shuffleGrouping("bolt1", TestDummyBolt.groupingStreamId)
.shuffleGrouping("bolt1", TestDummyBolt.shuffleStreamId)
- .fieldsGrouping("bolt2", TestDummyBolt.groupingStreamId, new Fields("id"))
+ .shuffleGrouping("bolt2", TestDummyBolt.groupingStreamId)
.shuffleGrouping("bolt2", TestDummyBolt.shuffleStreamId);
flinkBuilder.createTopology();
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-core/src/main/java/org/apache/flink/api/common/functions/AbstractRichFunction.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/AbstractRichFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/AbstractRichFunction.java
index 5a019aa..fd9de67 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/functions/AbstractRichFunction.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/AbstractRichFunction.java
@@ -38,10 +38,12 @@ public abstract class AbstractRichFunction implements RichFunction, Serializable
private transient RuntimeContext runtimeContext;
+ @Override
public void setRuntimeContext(RuntimeContext t) {
this.runtimeContext = t;
}
-
+
+ @Override
public RuntimeContext getRuntimeContext() {
if (this.runtimeContext != null) {
return this.runtimeContext;
@@ -50,6 +52,7 @@ public abstract class AbstractRichFunction implements RichFunction, Serializable
}
}
+ @Override
public IterationRuntimeContext getIterationRuntimeContext() {
if (this.runtimeContext == null) {
throw new IllegalStateException("The runtime context has not been initialized.");
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-core/src/main/java/org/apache/flink/api/common/functions/IterationRuntimeContext.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/IterationRuntimeContext.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/IterationRuntimeContext.java
index 73e738e..8239921 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/functions/IterationRuntimeContext.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/IterationRuntimeContext.java
@@ -22,7 +22,7 @@ import org.apache.flink.api.common.aggregators.Aggregator;
import org.apache.flink.types.Value;
/**
- *
+ *
*/
public interface IterationRuntimeContext extends RuntimeContext {
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFunction.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFunction.java
index 0685f63..0cbde4a 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFunction.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFunction.java
@@ -36,7 +36,7 @@ public interface RichFunction extends Function {
* The configuration contains all parameters that were configured on the function in the program
* composition.
*
- * <pre><blockquote>
+ * <pre>{@code
* public class MyMapper extends FilterFunction<String> {
*
* private String searchString;
@@ -49,7 +49,7 @@ public interface RichFunction extends Function {
* return value.equals(searchString);
* }
* }
- * </blockquote></pre>
+ * }</pre>
* <p>
* By default, this method does nothing.
*
@@ -64,7 +64,7 @@ public interface RichFunction extends Function {
void open(Configuration parameters) throws Exception;
/**
- * Teardown method for the user code. It is called after the last call to the main working methods
+ * Tear-down method for the user code. It is called after the last call to the main working methods
* (e.g. <i>map</i> or <i>join</i>). For functions that are part of an iteration, this method will
* be invoked after each iteration superstep.
* <p>
@@ -76,16 +76,32 @@ public interface RichFunction extends Function {
*/
void close() throws Exception;
+ // ------------------------------------------------------------------------
+ // Runtime context
+ // ------------------------------------------------------------------------
/**
- * Gets the context that contains information about the UDF's runtime.
+ * Gets the context that contains information about the UDF's runtime, such as the
+ * parallelism of the function, the subtask index of the function, or the name of
+ * the of the task that executes the function.
*
- * Context information are for example {@link org.apache.flink.api.common.accumulators.Accumulator}s
- * or the {@link org.apache.flink.api.common.cache.DistributedCache}.
+ * <p>The RuntimeContext also gives access to the
+ * {@link org.apache.flink.api.common.accumulators.Accumulator}s and the
+ * {@link org.apache.flink.api.common.cache.DistributedCache}.
*
* @return The UDF's runtime context.
*/
RuntimeContext getRuntimeContext();
+
+ /**
+ * Gets a specialized version of the {@link RuntimeContext}, which has additional information
+ * about the iteration in which the function is executed. This IterationRuntimeContext is only
+ * available if the function is part of an iteration. Otherwise, this method throws an exception.
+ *
+ * @return The IterationRuntimeContext.
+ * @throws java.lang.IllegalStateException Thrown, if the function is not executed as part of an iteration.
+ */
+ IterationRuntimeContext getIterationRuntimeContext();
/**
* Sets the function's runtime context. Called by the framework when creating a parallel instance of the function.
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java
index 289f063..cadef36 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java
@@ -18,7 +18,6 @@
package org.apache.flink.api.common.functions;
-import java.io.IOException;
import java.io.Serializable;
import java.util.List;
import java.util.Map;
@@ -31,7 +30,7 @@ import org.apache.flink.api.common.accumulators.IntCounter;
import org.apache.flink.api.common.accumulators.LongCounter;
import org.apache.flink.api.common.cache.DistributedCache;
import org.apache.flink.api.common.state.OperatorState;
-import org.apache.flink.api.common.state.StateCheckpointer;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
/**
* A RuntimeContext contains information about the context in which functions are executed. Each parallel instance
@@ -82,11 +81,7 @@ public interface RuntimeContext {
// --------------------------------------------------------------------------------------------
/**
- * Add this accumulator. Throws an exception if the counter is already
- * existing.
- *
- * This is only needed to support generic accumulators (e.g. for
- * Set<String>). Didn't find a way to get this work with getAccumulator.
+ * Add this accumulator. Throws an exception if the accumulator already exists.
*/
<V, A extends Serializable> void addAccumulator(String name, Accumulator<V, A> accumulator);
@@ -169,65 +164,101 @@ public interface RuntimeContext {
// --------------------------------------------------------------------------------------------
/**
- * Returns the {@link OperatorState} with the given name of the underlying
- * operator instance, which can be used to store and update user state in a
- * fault tolerant fashion. The state will be initialized by the provided
- * default value, and the {@link StateCheckpointer} will be used to draw the
- * state snapshots.
+ * Gets the key/value state, which is only accessible if the function is executed on
+ * a KeyedStream. Upon calling {@link OperatorState#value()}, the key/value state will
+ * return the value bound to the key of the element currently processed by the function.
+ *
+ * <p>Because the scope of each value is the key of the currently processed element,
+ * and the elements are distributed by the Flink runtime, the system can transparently
+ * scale out and redistribute the state and KeyedStream.
+ *
+ * <p>The following code example shows how to implement a continuous counter that counts
+ * how many times elements of a certain key occur, and emits an updated count for that
+ * element on each occurrence.
+ *
+ * <pre>{@code
+ * DataStream<MyType> stream = ...;
+ * KeyedStream<MyType> keyedStream = stream.keyBy("id");
+ *
+ * keyedStream.map(new RichMapFunction<MyType, Tuple2<MyType, Long>>() {
+ *
+ * private State<Long> state;
+ *
+ * public void open(Configuration cfg) {
+ * state = getRuntimeContext().getKeyValueState(Long.class, 0L);
+ * }
+ *
+ * public Tuple2<MyType, Long> map(MyType value) {
+ * long count = state.value();
+ * state.update(value + 1);
+ * return new Tuple2<>(value, count);
+ * }
+ * });
+ *
+ * }</pre>
+ *
+ * <p>This method attempts to deduce the type information from the given type class. If the
+ * full type cannot be determined from the class (for example because of generic parameters),
+ * the TypeInformation object must be manually passed via
+ * {@link #getKeyValueState(TypeInformation, Object)}.
*
- * <p>
- * When storing a {@link Serializable} state the user can omit the
- * {@link StateCheckpointer} in which case the full state will be written as
- * the snapshot.
- * </p>
- *
- * @param name
- * Identifier for the state allowing that more operator states
- * can be used by the same operator.
- * @param defaultState
- * Default value for the operator state. This will be returned
- * the first time {@link OperatorState#value()} (for every
- * state partition) is called before
- * {@link OperatorState#update(Object)}.
- * @param partitioned
- * Sets whether partitioning should be applied for the given
- * state. If true a partitioner key must be used.
- * @param checkpointer
- * The {@link StateCheckpointer} that will be used to draw
- * snapshots from the user state.
- * @return The {@link OperatorState} for the underlying operator.
- *
- * @throws IOException Thrown if the system cannot access the state.
- */
- <S, C extends Serializable> OperatorState<S> getOperatorState(String name, S defaultState,
- boolean partitioned, StateCheckpointer<S, C> checkpointer) throws IOException;
-
- /**
- * Returns the {@link OperatorState} with the given name of the underlying
- * operator instance, which can be used to store and update user state in a
- * fault tolerant fashion. The state will be initialized by the provided
- * default value.
+ * @param stateType The class of the type that is stored in the state. Used to generate
+ * serializers for managed memory and checkpointing.
+ * @param defaultState The default state value, returned when the state is accessed and
+ * no value has yet been set for the key. May be null.
+ * @param <S> The type of the state.
+ *
+ * @return The key/value state access.
+ *
+ * @throws UnsupportedOperationException Thrown, if no key/value state is available for the
+ * function (function is not part os a KeyedStream).
+ */
+ <S> OperatorState<S> getKeyValueState(Class<S> stateType, S defaultState);
+
+ /**
+ * Gets the key/value state, which is only accessible if the function is executed on
+ * a KeyedStream. Upon calling {@link OperatorState#value()}, the key/value state will
+ * return the value bound to the key of the element currently processed by the function.
*
- * <p>
- * When storing a non-{@link Serializable} state the user needs to specify a
- * {@link StateCheckpointer} for drawing snapshots.
- * </p>
- *
- * @param name
- * Identifier for the state allowing that more operator states
- * can be used by the same operator.
- * @param defaultState
- * Default value for the operator state. This will be returned
- * the first time {@link OperatorState#value()} (for every
- * state partition) is called before
- * {@link OperatorState#update(Object)}.
- * @param partitioned
- * Sets whether partitioning should be applied for the given
- * state. If true a partitioner key must be used.
- * @return The {@link OperatorState} for the underlying operator.
- *
- * @throws IOException Thrown if the system cannot access the state.
- */
- <S extends Serializable> OperatorState<S> getOperatorState(String name, S defaultState,
- boolean partitioned) throws IOException;
+ * <p>Because the scope of each value is the key of the currently processed element,
+ * and the elements are distributed by the Flink runtime, the system can transparently
+ * scale out and redistribute the state and KeyedStream.
+ *
+ * <p>The following code example shows how to implement a continuous counter that counts
+ * how many times elements of a certain key occur, and emits an updated count for that
+ * element on each occurrence.
+ *
+ * <pre>{@code
+ * DataStream<MyType> stream = ...;
+ * KeyedStream<MyType> keyedStream = stream.keyBy("id");
+ *
+ * keyedStream.map(new RichMapFunction<MyType, Tuple2<MyType, Long>>() {
+ *
+ * private State<Long> state;
+ *
+ * public void open(Configuration cfg) {
+ * state = getRuntimeContext().getKeyValueState(Long.class, 0L);
+ * }
+ *
+ * public Tuple2<MyType, Long> map(MyType value) {
+ * long count = state.value();
+ * state.update(value + 1);
+ * return new Tuple2<>(value, count);
+ * }
+ * });
+ *
+ * }</pre>
+ *
+ * @param stateType The type information for the type that is stored in the state.
+ * Used to create serializers for managed memory and checkpoints.
+ * @param defaultState The default state value, returned when the state is accessed and
+ * no value has yet been set for the key. May be null.
+ * @param <S> The type of the state.
+ *
+ * @return The key/value state access.
+ *
+ * @throws UnsupportedOperationException Thrown, if no key/value state is available for the
+ * function (function is not part os a KeyedStream).
+ */
+ <S> OperatorState<S> getKeyValueState(TypeInformation<S> stateType, S defaultState);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java
index 71be1e1..90d23cd 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java
@@ -18,7 +18,6 @@
package org.apache.flink.api.common.functions.util;
-import java.io.IOException;
import java.io.Serializable;
import java.util.Collections;
import java.util.Map;
@@ -35,7 +34,7 @@ import org.apache.flink.api.common.accumulators.LongCounter;
import org.apache.flink.api.common.cache.DistributedCache;
import org.apache.flink.api.common.functions.RuntimeContext;
import org.apache.flink.api.common.state.OperatorState;
-import org.apache.flink.api.common.state.StateCheckpointer;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.core.fs.Path;
/**
@@ -164,16 +163,16 @@ public abstract class AbstractRuntimeUDFContext implements RuntimeContext {
}
return (Accumulator<V, A>) accumulator;
}
-
+
@Override
- public <S, C extends Serializable> OperatorState<S> getOperatorState(String name,
- S defaultState, boolean partitioned, StateCheckpointer<S, C> checkpointer) throws IOException {
- throw new UnsupportedOperationException("Operator state is only accessible for streaming operators.");
+ public <S> OperatorState<S> getKeyValueState(Class<S> stateType, S defaultState) {
+ throw new UnsupportedOperationException(
+ "This state is only accessible by functions executed on a KeyedStream");
}
@Override
- public <S extends Serializable> OperatorState<S> getOperatorState(String name, S defaultState,
- boolean partitioned) throws IOException{
- throw new UnsupportedOperationException("Operator state is only accessible for streaming operators.");
+ public <S> OperatorState<S> getKeyValueState(TypeInformation<S> stateType, S defaultState) {
+ throw new UnsupportedOperationException(
+ "This state is only accessible by functions executed on a KeyedStream");
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/ListCollector.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/ListCollector.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/ListCollector.java
index a3a369b..12d9fda 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/ListCollector.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/ListCollector.java
@@ -22,6 +22,11 @@ import java.util.List;
import org.apache.flink.util.Collector;
+/**
+ * A {@link Collector} that puts the collected elements into a given list.
+ *
+ * @param <T> The type of the collected elements.
+ */
public class ListCollector<T> implements Collector<T> {
private final List<T> list;
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-core/src/main/java/org/apache/flink/api/common/state/OperatorState.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/OperatorState.java b/flink-core/src/main/java/org/apache/flink/api/common/state/OperatorState.java
index 3036023..136b6f8 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/state/OperatorState.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/state/OperatorState.java
@@ -20,24 +20,17 @@ package org.apache.flink.api.common.state;
import java.io.IOException;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.common.functions.AbstractRichFunction;
-import org.apache.flink.configuration.Configuration;
-
/**
- * Base interface for all streaming operator states. It can represent both
- * partitioned (when state partitioning is defined in the program) or
- * non-partitioned user states.
+ * This state interface abstracts persistent key/value state in streaming programs.
+ * The state is accessed and modified by user functions, and checkpointed consistently
+ * by the system as part of the distributed snapshots.
*
- * State can be accessed and manipulated using the {@link #value()} and
- * {@link #update(T)} methods. These calls are only safe in the
- * transformation call the operator represents, for instance inside
- * {@link MapFunction#map(Object)} and can lead tp unexpected behavior in the
- * {@link AbstractRichFunction#open(Configuration)} or
- * {@link AbstractRichFunction#close()} methods.
+ * <p>The state is only accessible by functions applied on a KeyedDataStream. The key is
+ * automatically supplied by the system, so the function always sees the value mapped to the
+ * key of the current element. That way, the system can handle stream and state partitioning
+ * consistently together.
*
- * @param <T>
- * Type of the operator state
+ * @param <T> Type of the value in the operator state
*/
public interface OperatorState<T> {
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-core/src/main/java/org/apache/flink/api/common/state/StateCheckpointer.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/StateCheckpointer.java b/flink-core/src/main/java/org/apache/flink/api/common/state/StateCheckpointer.java
deleted file mode 100644
index f373846..0000000
--- a/flink-core/src/main/java/org/apache/flink/api/common/state/StateCheckpointer.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * 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.api.common.state;
-
-import java.io.Serializable;
-
-/**
- * Basic interface for creating {@link OperatorState} snapshots in stateful
- * streaming programs.
- *
- * The user needs to implement the {@link #snapshotState(S, long, long)} and
- * {@link #restoreState(C)} methods that will be called to create and restore
- * state snapshots of the given states.
- *
- * <p>
- * Note that the {@link OperatorState} is <i>synchronously</i> checkpointed.
- * While the state is written, the state cannot be accessed or modified so the
- * function needs not return a copy of its state, but may return a reference to
- * its state.
- * </p>
- *
- * @param <S>
- * Type of the operator state.
- * @param <C>
- * Type of the snapshot that will be persisted.
- */
-public interface StateCheckpointer<S, C extends Serializable> {
-
- /**
- * Takes a snapshot of a given operator state. The snapshot returned will be
- * persisted in the state backend for this job and restored upon failure.
- * This method is called for all state partitions in case of partitioned
- * state when creating a checkpoint.
- *
- * @param state
- * The state for which the snapshot needs to be taken
- * @param checkpointId
- * The ID of the checkpoint.
- * @param checkpointTimestamp
- * The timestamp of the checkpoint, as derived by
- * System.currentTimeMillis() on the JobManager.
- *
- * @return A snapshot of the operator state.
- */
- C snapshotState(S state, long checkpointId, long checkpointTimestamp);
-
- /**
- * Restores the operator states from a given snapshot. The restores state
- * will be loaded back to the function. In case of partitioned state, each
- * partition is restored independently.
- *
- * @param stateSnapshot
- * The state snapshot that needs to be restored.
- * @return The state corresponding to the snapshot.
- */
- S restoreState(C stateSnapshot);
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
index 36369ab..b1ffdd8 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/ConfigConstants.java
@@ -415,11 +415,6 @@ public final class ConfigConstants {
*/
public static final String STATE_BACKEND = "state.backend";
- /**
- * Directory for saving streaming checkpoints
- */
- public static final String STATE_BACKEND_FS_DIR = "state.backend.fs.checkpointdir";
-
// ----------------------------- Miscellaneous ----------------------------
/**
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-core/src/main/java/org/apache/flink/core/memory/InputViewDataInputStreamWrapper.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/core/memory/InputViewDataInputStreamWrapper.java b/flink-core/src/main/java/org/apache/flink/core/memory/InputViewDataInputStreamWrapper.java
index 7de1d71..b4dffb1 100644
--- a/flink-core/src/main/java/org/apache/flink/core/memory/InputViewDataInputStreamWrapper.java
+++ b/flink-core/src/main/java/org/apache/flink/core/memory/InputViewDataInputStreamWrapper.java
@@ -121,9 +121,10 @@ public class InputViewDataInputStreamWrapper implements DataInputView, Closeable
public double readDouble() throws IOException {
return in.readDouble();
}
-
- @SuppressWarnings("deprecation")
+
@Override
+ @Deprecated
+ @SuppressWarnings("deprecation")
public String readLine() throws IOException {
return in.readLine();
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java b/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java
index de04dc4..8ce3e85 100644
--- a/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java
+++ b/flink-core/src/main/java/org/apache/flink/util/InstantiationUtil.java
@@ -246,7 +246,7 @@ public final class InstantiationUtil {
}
}
- public static Object readObjectFromConfig(Configuration config, String key, ClassLoader cl) throws IOException, ClassNotFoundException {
+ public static <T> T readObjectFromConfig(Configuration config, String key, ClassLoader cl) throws IOException, ClassNotFoundException {
byte[] bytes = config.getBytes(key, null);
if (bytes == null) {
return null;
@@ -284,13 +284,14 @@ public final class InstantiationUtil {
return serializer.deserialize(record, inputViewWrapper);
}
- public static Object deserializeObject(byte[] bytes, ClassLoader cl) throws IOException, ClassNotFoundException {
+ @SuppressWarnings("unchecked")
+ public static <T> T deserializeObject(byte[] bytes, ClassLoader cl) throws IOException, ClassNotFoundException {
ObjectInputStream oois = null;
final ClassLoader old = Thread.currentThread().getContextClassLoader();
try {
Thread.currentThread().setContextClassLoader(cl);
oois = new ClassLoaderObjectInputStream(new ByteArrayInputStream(bytes), cl);
- return oois.readObject();
+ return (T) oois.readObject();
} finally {
Thread.currentThread().setContextClassLoader(old);
if (oois != null) {
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-core/src/main/java/org/apache/flink/util/SerializedValue.java
----------------------------------------------------------------------
diff --git a/flink-core/src/main/java/org/apache/flink/util/SerializedValue.java b/flink-core/src/main/java/org/apache/flink/util/SerializedValue.java
index 5731fc1..504e458 100644
--- a/flink-core/src/main/java/org/apache/flink/util/SerializedValue.java
+++ b/flink-core/src/main/java/org/apache/flink/util/SerializedValue.java
@@ -55,6 +55,14 @@ public class SerializedValue<T> implements java.io.Serializable {
return serializedData == null ? null : (T) InstantiationUtil.deserializeObject(serializedData, loader);
}
+ /**
+ * Gets the size of the serialized state.
+ * @return The size of the serialized state.
+ */
+ public int getSizeOfSerializedState() {
+ return serializedData.length;
+ }
+
// --------------------------------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-core/src/test/java/org/apache/flink/core/testutils/CommonTestUtils.java
----------------------------------------------------------------------
diff --git a/flink-core/src/test/java/org/apache/flink/core/testutils/CommonTestUtils.java b/flink-core/src/test/java/org/apache/flink/core/testutils/CommonTestUtils.java
index 5b7afaa..4dbf04c 100644
--- a/flink-core/src/test/java/org/apache/flink/core/testutils/CommonTestUtils.java
+++ b/flink-core/src/test/java/org/apache/flink/core/testutils/CommonTestUtils.java
@@ -16,7 +16,6 @@
* limitations under the License.
*/
-
package org.apache.flink.core.testutils;
import static org.junit.Assert.fail;
@@ -37,8 +36,7 @@ import org.apache.flink.core.memory.InputViewDataInputStreamWrapper;
import org.apache.flink.core.memory.OutputViewDataOutputStreamWrapper;
/**
- * This class contains auxiliary methods for unit tests in the Nephele common module.
- *
+ * This class contains reusable utility methods for unit tests.
*/
public class CommonTestUtils {
@@ -127,9 +125,7 @@ public class CommonTestUtils {
T copy = null;
try {
copy = clazz.newInstance();
- } catch (InstantiationException e) {
- fail(e.getMessage());
- } catch (IllegalAccessException e) {
+ } catch (InstantiationException | IllegalAccessException e) {
fail(e.getMessage());
}
@@ -157,19 +153,14 @@ public class CommonTestUtils {
baos.close();
ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray());
- ObjectInputStream ois = new ObjectInputStream(bais);
- T copy;
- try {
- copy = (T) ois.readObject();
+ try (ObjectInputStream ois = new ObjectInputStream(bais)) {
+ @SuppressWarnings("unchecked")
+ T copy = (T) ois.readObject();
+ return copy;
}
catch (ClassNotFoundException e) {
throw new IOException(e);
}
-
- ois.close();
- bais.close();
-
- return copy;
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-dist/src/main/resources/flink-conf.yaml
----------------------------------------------------------------------
diff --git a/flink-dist/src/main/resources/flink-conf.yaml b/flink-dist/src/main/resources/flink-conf.yaml
index 4dd8173..1b04e35 100644
--- a/flink-dist/src/main/resources/flink-conf.yaml
+++ b/flink-dist/src/main/resources/flink-conf.yaml
@@ -79,16 +79,17 @@ webclient.port: 8080
# The backend that will be used to store operator state checkpoints if
# checkpointing is enabled.
#
-# Supported backends: jobmanager, filesystem
-
-state.backend: jobmanager
+# Supported backends: jobmanager, filesystem, <class-name-of-factory>
+#
+#state.backend: filesystem
-# Directory for storing checkpoints in a flink supported filesystem
-# Note: State backend must be accessible from the JobManager, use file://
-# only for local setups.
+# Directory for storing checkpoints in a Flink-supported filesystem
+# Note: State backend must be accessible from the JobManager and all TaskManagers.
+# Use "hdfs://" for HDFS setups, "file://" for UNIX/POSIX-compliant file systems,
+# (or any local file system under Windows), or "S3://" for S3 file system.
#
-# state.backend.fs.checkpointdir: hdfs://checkpoints
+# state.backend.fs.checkpointdir: hdfs://namenode-host:port/flink-checkpoints
#==============================================================================
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-runtime/src/main/java/org/apache/flink/runtime/state/FileStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/FileStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/FileStateHandle.java
index 091c739..c45990b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/FileStateHandle.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/FileStateHandle.java
@@ -67,35 +67,4 @@ public class FileStateHandle extends ByteStreamStateHandle {
public void discardState() throws Exception {
FileSystem.get(new URI(pathString)).delete(new Path(pathString), false);
}
-
- /**
- * Creates a {@link StateHandleProvider} for creating
- * {@link FileStateHandle}s for a given checkpoint directory.
- *
- */
- public static StateHandleProvider<Serializable> createProvider(String checkpointDir) {
- return new FileStateHandleProvider(checkpointDir);
- }
-
- /**
- * {@link StateHandleProvider} to generate {@link FileStateHandle}s for the
- * given checkpoint directory.
- *
- */
- private static class FileStateHandleProvider implements StateHandleProvider<Serializable> {
-
- private static final long serialVersionUID = 3496670017955260518L;
- private String path;
-
- public FileStateHandleProvider(String path) {
- this.path = path;
- }
-
- @Override
- public FileStateHandle createStateHandle(Serializable state) {
- return new FileStateHandle(state, path);
- }
-
- }
-
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalStateHandle.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalStateHandle.java
index 1b524d8..f2be70a 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalStateHandle.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/LocalStateHandle.java
@@ -40,18 +40,5 @@ public class LocalStateHandle<T extends Serializable> implements StateHandle<T>
}
@Override
- public void discardState() throws Exception {
- }
-
- public static class LocalStateHandleProvider<R extends Serializable> implements
- StateHandleProvider<R> {
-
- private static final long serialVersionUID = 4665419208932921425L;
-
- @Override
- public LocalStateHandle<R> createStateHandle(R state) {
- return new LocalStateHandle<R>(state);
- }
-
- }
+ public void discardState() {}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateHandleProvider.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateHandleProvider.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateHandleProvider.java
deleted file mode 100644
index bac490b..0000000
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateHandleProvider.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.runtime.state;
-
-import java.io.Serializable;
-
-/**
- * Stateful streaming operators use a StateHandleProvider to create new
- * {@link StateHandle}s to store each checkpoint in a persistent storage layer.
- */
-public interface StateHandleProvider<T> extends Serializable {
-
- /**
- * Creates a new {@link StateHandle} instance that will be used to store the
- * state checkpoint. This method is called for each state checkpoint saved.
- *
- * @param state
- * State to be stored in the handle.
- *
- */
- public StateHandle<T> createStateHandle(T state);
-
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
index 269222f..c8d50c7 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/taskmanager/Task.java
@@ -885,7 +885,6 @@ public class Task implements Runnable {
// build a local closure
final StatefulTask<?> statefulTask = (StatefulTask<?>) invokable;
- final Logger logger = LOG;
final String taskName = taskNameWithSubtask;
Runnable runnable = new Runnable() {
@@ -919,7 +918,6 @@ public class Task implements Runnable {
// build a local closure
final StatefulTask<?> statefulTask = (StatefulTask<?>) invokable;
- final Logger logger = LOG;
final String taskName = taskNameWithSubtask;
Runnable runnable = new Runnable() {
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-fs-tests/pom.xml
----------------------------------------------------------------------
diff --git a/flink-staging/flink-fs-tests/pom.xml b/flink-staging/flink-fs-tests/pom.xml
index fe1abb3..021d822 100644
--- a/flink-staging/flink-fs-tests/pom.xml
+++ b/flink-staging/flink-fs-tests/pom.xml
@@ -42,24 +42,42 @@ under the License.
<version>${project.version}</version>
<scope>test</scope>
</dependency>
+
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-core</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
+
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-streaming-core</artifactId>
+ <version>${project.version}</version>
+ <scope>test</scope>
+ </dependency>
+
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-java-examples</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
+
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-avro</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>
+
+ <dependency>
+ <groupId>org.apache.flink</groupId>
+ <artifactId>flink-test-utils</artifactId>
+ <version>${project.version}</version>
+ <scope>test</scope>
+ </dependency>
+
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-hdfs</artifactId>
@@ -67,6 +85,7 @@ under the License.
<type>test-jar</type>
<version>${hadoop.version}</version><!--$NO-MVN-MAN-VER$-->
</dependency>
+
<dependency>
<groupId>org.apache.hadoop</groupId>
<artifactId>hadoop-common</artifactId>
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/FileStateBackendTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/FileStateBackendTest.java b/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/FileStateBackendTest.java
new file mode 100644
index 0000000..8b7fb1c
--- /dev/null
+++ b/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/FileStateBackendTest.java
@@ -0,0 +1,308 @@
+/*
+ * 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.hdfstests;
+
+import org.apache.commons.io.FileUtils;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.ConfigConstants;
+import org.apache.flink.core.fs.FileStatus;
+import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.core.testutils.CommonTestUtils;
+import org.apache.flink.runtime.state.StateHandle;
+import org.apache.flink.streaming.api.state.filesystem.FileStreamStateHandle;
+import org.apache.flink.streaming.api.state.filesystem.FsStateBackend;
+
+import org.apache.flink.streaming.api.state.StateBackend;
+import org.apache.flink.streaming.api.state.StreamStateHandle;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URI;
+import java.util.Random;
+import java.util.UUID;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class FileStateBackendTest {
+
+ private static File TEMP_DIR;
+
+ private static String HDFS_ROOT_URI;
+
+ private static MiniDFSCluster HDFS_CLUSTER;
+
+ private static FileSystem FS;
+
+ // ------------------------------------------------------------------------
+ // startup / shutdown
+ // ------------------------------------------------------------------------
+
+ @BeforeClass
+ public static void createHDFS() {
+ try {
+ TEMP_DIR = new File(ConfigConstants.DEFAULT_TASK_MANAGER_TMP_PATH, UUID.randomUUID().toString());
+
+ Configuration hdConf = new Configuration();
+ hdConf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, TEMP_DIR.getAbsolutePath());
+ MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(hdConf);
+ HDFS_CLUSTER = builder.build();
+
+ HDFS_ROOT_URI = "hdfs://" + HDFS_CLUSTER.getURI().getHost() + ":"
+ + HDFS_CLUSTER.getNameNodePort() + "/";
+
+ FS = FileSystem.get(new URI(HDFS_ROOT_URI));
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail("Could not create HDFS mini cluster " + e.getMessage());
+ }
+ }
+
+ @AfterClass
+ public static void destroyHDFS() {
+ try {
+ HDFS_CLUSTER.shutdown();
+ FileUtils.deleteDirectory(TEMP_DIR);
+ }
+ catch (Exception ignored) {}
+ }
+
+ // ------------------------------------------------------------------------
+ // Tests
+ // ------------------------------------------------------------------------
+
+ @Test
+ public void testSetupAndSerialization() {
+ try {
+ URI baseUri = new URI(HDFS_ROOT_URI + UUID.randomUUID().toString());
+
+ FsStateBackend originalBackend = new FsStateBackend(baseUri);
+
+ assertFalse(originalBackend.isInitialized());
+ assertEquals(baseUri, originalBackend.getBasePath().toUri());
+ assertNull(originalBackend.getCheckpointDirectory());
+
+ // serialize / copy the backend
+ FsStateBackend backend = CommonTestUtils.createCopySerializable(originalBackend);
+ assertFalse(backend.isInitialized());
+ assertEquals(baseUri, backend.getBasePath().toUri());
+ assertNull(backend.getCheckpointDirectory());
+
+ // no file operations should be possible right now
+ try {
+ backend.checkpointStateSerializable("exception train rolling in", 2L, System.currentTimeMillis());
+ fail("should fail with an exception");
+ } catch (IllegalStateException e) {
+ // supreme!
+ }
+
+ backend.initializeForJob(new JobID());
+ assertNotNull(backend.getCheckpointDirectory());
+
+ Path checkpointDir = backend.getCheckpointDirectory();
+ assertTrue(FS.exists(checkpointDir));
+ assertTrue(isDirectoryEmpty(checkpointDir));
+
+ backend.disposeAllStateForCurrentJob();
+ assertNull(backend.getCheckpointDirectory());
+
+ assertTrue(isDirectoryEmpty(baseUri));
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testSerializableState() {
+
+ try {
+ FsStateBackend backend = CommonTestUtils.createCopySerializable(new FsStateBackend(randomHdfsFileUri()));
+ backend.initializeForJob(new JobID());
+
+ Path checkpointDir = backend.getCheckpointDirectory();
+
+ String state1 = "dummy state";
+ String state2 = "row row row your boat";
+ Integer state3 = 42;
+
+ StateHandle<String> handle1 = backend.checkpointStateSerializable(state1, 439568923746L, System.currentTimeMillis());
+ StateHandle<String> handle2 = backend.checkpointStateSerializable(state2, 439568923746L, System.currentTimeMillis());
+ StateHandle<Integer> handle3 = backend.checkpointStateSerializable(state3, 439568923746L, System.currentTimeMillis());
+
+ assertFalse(isDirectoryEmpty(checkpointDir));
+ assertEquals(state1, handle1.getState(getClass().getClassLoader()));
+ handle1.discardState();
+
+ assertFalse(isDirectoryEmpty(checkpointDir));
+ assertEquals(state2, handle2.getState(getClass().getClassLoader()));
+ handle2.discardState();
+
+ assertFalse(isDirectoryEmpty(checkpointDir));
+ assertEquals(state3, handle3.getState(getClass().getClassLoader()));
+ handle3.discardState();
+
+ assertTrue(isDirectoryEmpty(checkpointDir));
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ @Test
+ public void testStateOutputStream() {
+ try {
+ FsStateBackend backend = CommonTestUtils.createCopySerializable(new FsStateBackend(randomHdfsFileUri()));
+ backend.initializeForJob(new JobID());
+
+ Path checkpointDir = backend.getCheckpointDirectory();
+
+ byte[] state1 = new byte[1274673];
+ byte[] state2 = new byte[1];
+ byte[] state3 = new byte[0];
+ byte[] state4 = new byte[177];
+
+ Random rnd = new Random();
+ rnd.nextBytes(state1);
+ rnd.nextBytes(state2);
+ rnd.nextBytes(state3);
+ rnd.nextBytes(state4);
+
+ long checkpointId = 97231523452L;
+
+ FsStateBackend.FsCheckpointStateOutputStream stream1 =
+ backend.createCheckpointStateOutputStream(checkpointId, System.currentTimeMillis());
+ FsStateBackend.FsCheckpointStateOutputStream stream2 =
+ backend.createCheckpointStateOutputStream(checkpointId, System.currentTimeMillis());
+ FsStateBackend.FsCheckpointStateOutputStream stream3 =
+ backend.createCheckpointStateOutputStream(checkpointId, System.currentTimeMillis());
+
+ stream1.write(state1);
+ stream2.write(state2);
+ stream3.write(state3);
+
+ FileStreamStateHandle handle1 = stream1.closeAndGetHandle();
+ FileStreamStateHandle handle2 = stream2.closeAndGetHandle();
+ FileStreamStateHandle handle3 = stream3.closeAndGetHandle();
+
+ // use with try-with-resources
+ StreamStateHandle handle4;
+ try (StateBackend.CheckpointStateOutputStream stream4 =
+ backend.createCheckpointStateOutputStream(checkpointId, System.currentTimeMillis())) {
+ stream4.write(state4);
+ handle4 = stream4.closeAndGetHandle();
+ }
+
+ // close before accessing handle
+ StateBackend.CheckpointStateOutputStream stream5 =
+ backend.createCheckpointStateOutputStream(checkpointId, System.currentTimeMillis());
+ stream5.write(state4);
+ stream5.close();
+ try {
+ stream5.closeAndGetHandle();
+ fail();
+ } catch (IOException e) {
+ // uh-huh
+ }
+
+ validateBytesInStream(handle1.getState(getClass().getClassLoader()), state1);
+ handle1.discardState();
+ assertFalse(isDirectoryEmpty(checkpointDir));
+ ensureFileDeleted(handle1.getFilePath());
+
+ validateBytesInStream(handle2.getState(getClass().getClassLoader()), state2);
+ handle2.discardState();
+ assertFalse(isDirectoryEmpty(checkpointDir));
+ ensureFileDeleted(handle2.getFilePath());
+
+ validateBytesInStream(handle3.getState(getClass().getClassLoader()), state3);
+ handle3.discardState();
+ assertFalse(isDirectoryEmpty(checkpointDir));
+ ensureFileDeleted(handle3.getFilePath());
+
+ validateBytesInStream(handle4.getState(getClass().getClassLoader()), state4);
+ handle4.discardState();
+ assertTrue(isDirectoryEmpty(checkpointDir));
+ }
+ catch (Exception e) {
+ e.printStackTrace();
+ fail(e.getMessage());
+ }
+ }
+
+ // ------------------------------------------------------------------------
+ // Utilities
+ // ------------------------------------------------------------------------
+
+ private static void ensureFileDeleted(Path path) {
+ try {
+ assertFalse(FS.exists(path));
+ }
+ catch (IOException ignored) {}
+ }
+
+ private static boolean isDirectoryEmpty(URI directory) {
+ return isDirectoryEmpty(new Path(directory));
+ }
+
+ private static boolean isDirectoryEmpty(Path directory) {
+ try {
+ FileStatus[] nested = FS.listStatus(directory);
+ return nested == null || nested.length == 0;
+ }
+ catch (IOException e) {
+ return true;
+ }
+ }
+
+ private static String randomHdfsFileUri() {
+ return HDFS_ROOT_URI + UUID.randomUUID().toString();
+ }
+
+ private static void validateBytesInStream(InputStream is, byte[] data) throws IOException {
+ byte[] holder = new byte[data.length];
+
+ int pos = 0;
+ int read;
+ while (pos < holder.length && (read = is.read(holder, pos, holder.length - pos)) != -1) {
+ pos += read;
+ }
+
+ assertEquals("not enough data", holder.length, pos);
+ assertEquals("too much data", -1, is.read());
+ assertArrayEquals("wrong data", data, holder);
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/FileStateHandleTest.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/FileStateHandleTest.java b/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/FileStateHandleTest.java
deleted file mode 100644
index 59ee5a9..0000000
--- a/flink-staging/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/FileStateHandleTest.java
+++ /dev/null
@@ -1,126 +0,0 @@
-/*
- * 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.hdfstests;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.Serializable;
-
-import org.apache.flink.runtime.state.FileStateHandle;
-import org.apache.flink.runtime.state.StateHandle;
-import org.apache.flink.runtime.state.StateHandleProvider;
-import org.apache.flink.util.SerializedValue;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-public class FileStateHandleTest {
-
- private String hdfsURI;
- private MiniDFSCluster hdfsCluster;
- private org.apache.hadoop.fs.Path hdPath;
- private org.apache.hadoop.fs.FileSystem hdfs;
-
- @Before
- public void createHDFS() {
- try {
- Configuration hdConf = new Configuration();
-
- File baseDir = new File("./target/hdfs/filestatehandletest").getAbsoluteFile();
- FileUtil.fullyDelete(baseDir);
- hdConf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath());
- MiniDFSCluster.Builder builder = new MiniDFSCluster.Builder(hdConf);
- hdfsCluster = builder.build();
-
- hdfsURI = "hdfs://" + hdfsCluster.getURI().getHost() + ":"
- + hdfsCluster.getNameNodePort() + "/";
-
- hdPath = new org.apache.hadoop.fs.Path("/StateHandleTest");
- hdfs = hdPath.getFileSystem(hdConf);
- hdfs.mkdirs(hdPath);
-
- } catch (Throwable e) {
- e.printStackTrace();
- Assert.fail("Test failed " + e.getMessage());
- }
- }
-
- @After
- public void destroyHDFS() {
- try {
- hdfs.delete(hdPath, true);
- hdfsCluster.shutdown();
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
-
- }
-
- @Test
- public void testFileStateHandle() throws Exception {
-
- Serializable state = "state";
-
- // Create a state handle provider for the hdfs directory
- StateHandleProvider<Serializable> handleProvider = FileStateHandle.createProvider(hdfsURI
- + hdPath);
-
- FileStateHandle handle = (FileStateHandle) handleProvider.createStateHandle(state);
-
- try {
- handleProvider.createStateHandle(null);
- fail();
- } catch (RuntimeException e) {
- // good
- }
-
- assertTrue(handle.stateFetched());
- assertFalse(handle.isWritten());
-
- // Serialize the handle so it writes the value to hdfs
- SerializedValue<StateHandle<Serializable>> serializedHandle = new SerializedValue<StateHandle<Serializable>>(
- handle);
-
- assertTrue(handle.isWritten());
-
- // Deserialize the handle and verify that the state is not fetched yet
- FileStateHandle deserializedHandle = (FileStateHandle) serializedHandle
- .deserializeValue(Thread.currentThread().getContextClassLoader());
- assertFalse(deserializedHandle.stateFetched());
-
- // Fetch the and compare with original
- assertEquals(state, deserializedHandle.getState(this.getClass().getClassLoader()));
-
- // Test whether discard removes the checkpoint file properly
- assertTrue(hdfs.listFiles(hdPath, true).hasNext());
- deserializedHandle.discardState();
- assertFalse(hdfs.listFiles(hdPath, true).hasNext());
-
- }
-
-}
[22/24] flink git commit: [hotfix] [streaming scala] Expose key type
information for key selectors on connected data streams
Posted by se...@apache.org.
[hotfix] [streaming scala] Expose key type information for key selectors on connected data streams
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/bb1f5fd5
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/bb1f5fd5
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/bb1f5fd5
Branch: refs/heads/master
Commit: bb1f5fd58c5a371941531740d573300ab020503b
Parents: 4ee5b4c
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Oct 9 23:58:40 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 16 15:26:11 2015 +0200
----------------------------------------------------------------------
.../streaming/api/scala/ConnectedStreams.scala | 30 ++++++++++++++------
1 file changed, 21 insertions(+), 9 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/bb1f5fd5/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedStreams.scala
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedStreams.scala b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedStreams.scala
index 4727cc5..f7413b7 100644
--- a/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedStreams.scala
+++ b/flink-staging/flink-streaming/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/ConnectedStreams.scala
@@ -20,7 +20,8 @@ package org.apache.flink.streaming.api.scala
import org.apache.flink.api.common.typeinfo.TypeInformation
import org.apache.flink.api.java.functions.KeySelector
-import org.apache.flink.streaming.api.datastream.{ConnectedStreams => JavaCStream, DataStream => JavaStream}
+import org.apache.flink.api.java.typeutils.{TypeExtractor, ResultTypeQueryable}
+import org.apache.flink.streaming.api.datastream.{ConnectedStreams => JavaCStream, DataStream => JavaStream, KeyedStream => JKeyedStream}
import org.apache.flink.streaming.api.functions.co.{CoFlatMapFunction, CoMapFunction}
import org.apache.flink.util.Collector
import scala.reflect.ClassTag
@@ -238,18 +239,18 @@ class ConnectedStreams[IN1, IN2](javaStream: JavaCStream[IN1, IN2]) {
* The function used for grouping the second input
* @return The grouped { @link ConnectedStreams}
*/
- def keyBy[K: TypeInformation, L: TypeInformation](fun1: IN1 => K, fun2: IN2 => L):
+ def keyBy[K1: TypeInformation, K2: TypeInformation](fun1: IN1 => K1, fun2: IN2 => K2):
ConnectedStreams[IN1, IN2] = {
+ val keyType1 = implicitly[TypeInformation[K1]]
+ val keyType2 = implicitly[TypeInformation[K2]]
+
val cleanFun1 = clean(fun1)
val cleanFun2 = clean(fun2)
- val keyExtractor1 = new KeySelector[IN1, K] {
- def getKey(in: IN1) = cleanFun1(in)
- }
- val keyExtractor2 = new KeySelector[IN2, L] {
- def getKey(in: IN2) = cleanFun2(in)
- }
-
+
+ val keyExtractor1 = new KeySelectorWithType[IN1, K1](cleanFun1, keyType1)
+ val keyExtractor2 = new KeySelectorWithType[IN2, K2](cleanFun2, keyType2)
+
javaStream.keyBy(keyExtractor1, keyExtractor2)
}
@@ -351,3 +352,14 @@ class ConnectedStreams[IN1, IN2](javaStream: JavaCStream[IN1, IN2]) {
}
}
+
+class KeySelectorWithType[IN, K](
+ private[this] val fun: IN => K,
+ private[this] val info: TypeInformation[K])
+ extends KeySelector[IN, K] with ResultTypeQueryable[K] {
+
+ override def getKey(value: IN): K = fun(value)
+
+ override def getProducedType: TypeInformation[K] = info
+}
+
\ No newline at end of file
[06/24] flink git commit: [FLINK-2808] [streaming] Refactor and
extend state backend abstraction
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
index 80563b8..e131cda 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/io/StreamInputProcessor.java
@@ -42,7 +42,6 @@ import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordS
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer;
import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
-import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext;
/**
* Input reader for {@link org.apache.flink.streaming.runtime.tasks.OneInputStreamTask}.
@@ -125,7 +124,8 @@ public class StreamInputProcessor<IN> {
lastEmittedWatermark = Long.MIN_VALUE;
}
- public boolean processInput(OneInputStreamOperator<IN, ?> streamOperator, Object lock) throws Exception {
+ @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+ public boolean processInput(OneInputStreamOperator<IN, ?> streamOperator, final Object lock) throws Exception {
if (isFinished) {
return false;
}
@@ -161,11 +161,8 @@ public class StreamInputProcessor<IN> {
} else {
// now we can do the actual processing
StreamRecord<IN> record = recordOrWatermark.asRecord();
- StreamingRuntimeContext ctx = streamOperator.getRuntimeContext();
synchronized (lock) {
- if (ctx != null) {
- ctx.setNextInput(record);
- }
+ streamOperator.setKeyContextElement(record);
streamOperator.processElement(record);
}
return true;
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/BucketStreamSortOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/BucketStreamSortOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/BucketStreamSortOperator.java
index 017c8ea..7020758 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/BucketStreamSortOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/BucketStreamSortOperator.java
@@ -17,17 +17,16 @@
*/
package org.apache.flink.streaming.runtime.operators;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
@@ -57,10 +56,9 @@ public class BucketStreamSortOperator<T> extends AbstractStreamOperator<T> imple
}
@Override
- @SuppressWarnings("unchecked")
- public void open(Configuration parameters) throws Exception {
- super.open(parameters);
- buckets = Maps.newHashMap();
+ public void open() throws Exception {
+ super.open();
+ buckets = new HashMap<>();
}
@@ -70,7 +68,7 @@ public class BucketStreamSortOperator<T> extends AbstractStreamOperator<T> imple
long bucketId = record.getTimestamp() - (record.getTimestamp() % granularity);
List<StreamRecord<T>> bucket = buckets.get(bucketId);
if (bucket == null) {
- bucket = Lists.newArrayList();
+ bucket = new ArrayList<>();
buckets.put(bucketId, bucket);
}
bucket.add(record);
@@ -79,7 +77,7 @@ public class BucketStreamSortOperator<T> extends AbstractStreamOperator<T> imple
@Override
public void processWatermark(Watermark mark) throws Exception {
long maxBucketId = mark.getTimestamp() - (mark.getTimestamp() % granularity);
- Set<Long> toRemove = Sets.newHashSet();
+ Set<Long> toRemove = new HashSet<>();
for (Map.Entry<Long, List<StreamRecord<T>>> bucket: buckets.entrySet()) {
if (bucket.getKey() < maxBucketId) {
Collections.sort(bucket.getValue(), new Comparator<StreamRecord<T>>() {
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/ExtractTimestampsOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/ExtractTimestampsOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/ExtractTimestampsOperator.java
index ddfc6a1..6e51a49 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/ExtractTimestampsOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/ExtractTimestampsOperator.java
@@ -17,9 +17,9 @@
package org.apache.flink.streaming.runtime.operators;
-import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.functions.TimestampExtractor;
import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
@@ -46,11 +46,11 @@ public class ExtractTimestampsOperator<T>
}
@Override
- public void open(Configuration parameters) throws Exception {
- super.open(parameters);
- watermarkInterval = getRuntimeContext().getExecutionConfig().getAutoWatermarkInterval();
+ public void open() throws Exception {
+ super.open();
+ watermarkInterval = getExecutionConfig().getAutoWatermarkInterval();
if (watermarkInterval > 0) {
- getRuntimeContext().registerTimer(System.currentTimeMillis() + watermarkInterval, this);
+ registerTimer(System.currentTimeMillis() + watermarkInterval, this);
}
currentWatermark = Long.MIN_VALUE;
@@ -78,7 +78,7 @@ public class ExtractTimestampsOperator<T>
@Override
public void trigger(long timestamp) throws Exception {
// register next timer
- getRuntimeContext().registerTimer(System.currentTimeMillis() + watermarkInterval, this);
+ registerTimer(System.currentTimeMillis() + watermarkInterval, this);
long lastWatermark = currentWatermark;
currentWatermark = userFunction.getCurrentWatermark();
@@ -90,6 +90,6 @@ public class ExtractTimestampsOperator<T>
@Override
public void processWatermark(Watermark mark) throws Exception {
- // ingore them, since we are basically a watermark source
+ // ignore them, since we are basically a watermark source
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractAlignedProcessingTimeWindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractAlignedProcessingTimeWindowOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractAlignedProcessingTimeWindowOperator.java
index 4fcfb2c..227de49 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractAlignedProcessingTimeWindowOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/AbstractAlignedProcessingTimeWindowOperator.java
@@ -22,7 +22,6 @@ import org.apache.commons.math3.util.ArithmeticUtils;
import org.apache.flink.api.common.functions.Function;
import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.util.MathUtils;
import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
@@ -105,8 +104,8 @@ public abstract class AbstractAlignedProcessingTimeWindowOperator<KEY, IN, OUT,
// ------------------------------------------------------------------------
@Override
- public void open(Configuration parameters) throws Exception {
- super.open(parameters);
+ public void open() throws Exception {
+ super.open();
out = new TimestampedCollector<>(output);
@@ -119,7 +118,7 @@ public abstract class AbstractAlignedProcessingTimeWindowOperator<KEY, IN, OUT,
nextEvaluationTime = now + windowSlide - (now % windowSlide);
nextSlideTime = now + paneSize - (now % paneSize);
- getRuntimeContext().registerTimer(Math.min(nextEvaluationTime, nextSlideTime), this);
+ registerTimer(Math.min(nextEvaluationTime, nextSlideTime), this);
}
@Override
@@ -188,7 +187,7 @@ public abstract class AbstractAlignedProcessingTimeWindowOperator<KEY, IN, OUT,
}
long nextTriggerTime = Math.min(nextEvaluationTime, nextSlideTime);
- getRuntimeContext().registerTimer(nextTriggerTime, this);
+ registerTimer(nextTriggerTime, this);
}
private void computeWindow(long timestamp) throws Exception {
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperator.java
index f35ffca..aecfd5d 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/NonKeyedWindowOperator.java
@@ -18,16 +18,15 @@
package org.apache.flink.streaming.runtime.operators.windowing;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
+
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.typeutils.InputTypeConfigurable;
-import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.operators.OutputTypeConfigurable;
import org.apache.flink.streaming.api.operators.TimestampedCollector;
@@ -39,10 +38,13 @@ import org.apache.flink.streaming.runtime.operators.Triggerable;
import org.apache.flink.streaming.runtime.operators.windowing.buffers.WindowBuffer;
import org.apache.flink.streaming.runtime.operators.windowing.buffers.WindowBufferFactory;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
import java.util.Map;
import java.util.Set;
@@ -104,11 +106,11 @@ public class NonKeyedWindowOperator<IN, OUT, W extends Window>
}
@Override
- public void open(Configuration parameters) throws Exception {
- super.open(parameters);
- windows = Maps.newHashMap();
- watermarkTimers = Maps.newHashMap();
- processingTimeTimers = Maps.newHashMap();
+ public void open() throws Exception {
+ super.open();
+ windows = new HashMap<>();
+ watermarkTimers = new HashMap<>();
+ processingTimeTimers = new HashMap<>();
timestampedCollector = new TimestampedCollector<>(output);
if (inputSerializer == null) {
@@ -116,7 +118,7 @@ public class NonKeyedWindowOperator<IN, OUT, W extends Window>
}
windowBufferFactory.setRuntimeContext(getRuntimeContext());
- windowBufferFactory.open(parameters);
+ windowBufferFactory.open(getUserFunctionParameters());
}
@Override
@@ -192,7 +194,7 @@ public class NonKeyedWindowOperator<IN, OUT, W extends Window>
@Override
public void processWatermark(Watermark mark) throws Exception {
- Set<Long> toRemove = Sets.newHashSet();
+ Set<Long> toRemove = new HashSet<>();
for (Map.Entry<Long, Set<TriggerContext>> triggers: watermarkTimers.entrySet()) {
if (triggers.getKey() <= mark.getTimestamp()) {
@@ -212,7 +214,7 @@ public class NonKeyedWindowOperator<IN, OUT, W extends Window>
@Override
public void trigger(long time) throws Exception {
- Set<Long> toRemove = Sets.newHashSet();
+ Set<Long> toRemove = new HashSet<>();
for (Map.Entry<Long, Set<TriggerContext>> triggers: processingTimeTimers.entrySet()) {
if (triggers.getKey() < time) {
@@ -243,7 +245,7 @@ public class NonKeyedWindowOperator<IN, OUT, W extends Window>
Set<TriggerContext> triggers = processingTimeTimers.get(time);
if (triggers == null) {
getRuntimeContext().registerTimer(time, NonKeyedWindowOperator.this);
- triggers = Sets.newHashSet();
+ triggers = new HashSet<>();
processingTimeTimers.put(time, triggers);
}
triggers.add(this);
@@ -253,7 +255,7 @@ public class NonKeyedWindowOperator<IN, OUT, W extends Window>
public void registerWatermarkTimer(long time) {
Set<TriggerContext> triggers = watermarkTimers.get(time);
if (triggers == null) {
- triggers = Sets.newHashSet();
+ triggers = new HashSet<>();
watermarkTimers.put(time, triggers);
}
triggers.add(this);
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
index da36db1..82a3f9a 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java
@@ -18,17 +18,16 @@
package org.apache.flink.streaming.runtime.operators.windowing;
import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
+
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.typeutils.InputTypeConfigurable;
-import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.operators.OutputTypeConfigurable;
import org.apache.flink.streaming.api.operators.TimestampedCollector;
@@ -44,6 +43,8 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
import java.util.Map;
import java.util.Set;
@@ -149,11 +150,11 @@ public class WindowOperator<K, IN, OUT, W extends Window>
}
@Override
- public void open(Configuration parameters) throws Exception {
- super.open(parameters);
- windows = Maps.newHashMap();
- watermarkTimers = Maps.newHashMap();
- processingTimeTimers = Maps.newHashMap();
+ public void open() throws Exception {
+ super.open();
+ windows = new HashMap<>();
+ watermarkTimers = new HashMap<>();
+ processingTimeTimers = new HashMap<>();
timestampedCollector = new TimestampedCollector<>(output);
if (inputSerializer == null) {
@@ -161,7 +162,7 @@ public class WindowOperator<K, IN, OUT, W extends Window>
}
windowBufferFactory.setRuntimeContext(getRuntimeContext());
- windowBufferFactory.open(parameters);
+ windowBufferFactory.open(getUserFunctionParameters());
}
@Override
@@ -191,7 +192,7 @@ public class WindowOperator<K, IN, OUT, W extends Window>
Map<W, Tuple2<WindowBuffer<IN>, TriggerContext>> keyWindows = windows.get(key);
if (keyWindows == null) {
- keyWindows = Maps.newHashMap();
+ keyWindows = new HashMap<>();
windows.put(key, keyWindows);
}
@@ -260,7 +261,7 @@ public class WindowOperator<K, IN, OUT, W extends Window>
@Override
public void processWatermark(Watermark mark) throws Exception {
- Set<Long> toRemove = Sets.newHashSet();
+ Set<Long> toRemove = new HashSet<>();
for (Map.Entry<Long, Set<TriggerContext>> triggers: watermarkTimers.entrySet()) {
if (triggers.getKey() <= mark.getTimestamp()) {
@@ -280,7 +281,7 @@ public class WindowOperator<K, IN, OUT, W extends Window>
@Override
public void trigger(long time) throws Exception {
- Set<Long> toRemove = Sets.newHashSet();
+ Set<Long> toRemove = new HashSet<>();
for (Map.Entry<Long, Set<TriggerContext>> triggers: processingTimeTimers.entrySet()) {
if (triggers.getKey() < time) {
@@ -317,7 +318,7 @@ public class WindowOperator<K, IN, OUT, W extends Window>
Set<TriggerContext> triggers = processingTimeTimers.get(time);
if (triggers == null) {
getRuntimeContext().registerTimer(time, WindowOperator.this);
- triggers = Sets.newHashSet();
+ triggers = new HashSet<>();
processingTimeTimers.put(time, triggers);
}
triggers.add(this);
@@ -327,7 +328,7 @@ public class WindowOperator<K, IN, OUT, W extends Window>
public void registerWatermarkTimer(long time) {
Set<TriggerContext> triggers = watermarkTimers.get(time);
if (triggers == null) {
- triggers = Sets.newHashSet();
+ triggers = new HashSet<>();
watermarkTimers.put(time, triggers);
}
triggers.add(this);
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/ExceptionInChainedOperatorException.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/ExceptionInChainedOperatorException.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/ExceptionInChainedOperatorException.java
new file mode 100644
index 0000000..ec90bff
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/ExceptionInChainedOperatorException.java
@@ -0,0 +1,45 @@
+/*
+ * 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.runtime.tasks;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * A special exception that signifies that the cause exception came from a chained operator.
+ */
+public class ExceptionInChainedOperatorException extends RuntimeException {
+
+ private static final long serialVersionUID = 1L;
+
+ public ExceptionInChainedOperatorException(Throwable cause) {
+ this("Could not forward element to next operator", cause);
+ }
+
+ public ExceptionInChainedOperatorException(String message, Throwable cause) {
+ super(message, requireNonNull(cause));
+ }
+
+ public Throwable getOriginalCause() {
+ Throwable ex = this;
+ do {
+ ex = ex.getCause();
+ } while (ex instanceof ExceptionInChainedOperatorException);
+ return ex;
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java
index 89eac92..5316ae4 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTask.java
@@ -21,6 +21,7 @@ package org.apache.flink.streaming.runtime.tasks;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
import org.apache.flink.runtime.io.network.partition.consumer.InputGate;
+import org.apache.flink.streaming.api.graph.StreamConfig;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.runtime.io.StreamInputProcessor;
@@ -32,6 +33,8 @@ public class OneInputStreamTask<IN, OUT> extends StreamTask<OUT, OneInputStreamO
@Override
public void init() throws Exception {
+ StreamConfig configuration = getConfiguration();
+
TypeSerializer<IN> inSerializer = configuration.getTypeSerializerIn1(getUserCodeClassLoader());
int numberOfInputs = configuration.getNumberOfInputs();
@@ -52,10 +55,13 @@ public class OneInputStreamTask<IN, OUT> extends StreamTask<OUT, OneInputStreamO
@Override
protected void run() throws Exception {
- while (running && inputProcessor.processInput(streamOperator, lock)) {
- if (timerException != null) {
- throw timerException;
- }
+ // cache some references on the stack, to make the code more JIT friendly
+ final OneInputStreamOperator<IN, OUT> operator = this.headOperator;
+ final StreamInputProcessor<IN> inputProcessor = this.inputProcessor;
+ final Object lock = getCheckpointLock();
+
+ while (running && inputProcessor.processInput(operator, lock)) {
+ checkTimerException();
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java
new file mode 100644
index 0000000..9df3a5d
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java
@@ -0,0 +1,308 @@
+/*
+ * 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.runtime.tasks;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
+import org.apache.flink.runtime.execution.Environment;
+import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
+import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
+import org.apache.flink.runtime.plugable.SerializationDelegate;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.streaming.runtime.io.CollectorWrapper;
+import org.apache.flink.streaming.runtime.io.RecordWriterOutput;
+import org.apache.flink.streaming.api.collector.selector.OutputSelectorWrapper;
+import org.apache.flink.streaming.api.graph.StreamConfig;
+import org.apache.flink.streaming.api.graph.StreamEdge;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.api.operators.Output;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.runtime.io.StreamRecordWriter;
+import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class OperatorChain<OUT> {
+
+ private static final Logger LOG = LoggerFactory.getLogger(OperatorChain.class);
+
+ private final StreamOperator<?>[] allOperators;
+
+ private final RecordWriterOutput<?>[] streamOutputs;
+
+ private final Output<StreamRecord<OUT>> chainEntryPoint;
+
+
+ public OperatorChain(StreamTask<OUT, ?> containingTask,
+ StreamOperator<OUT> headOperator,
+ AccumulatorRegistry.Reporter reporter) {
+
+ final ClassLoader userCodeClassloader = containingTask.getUserCodeClassLoader();
+ final StreamConfig configuration = containingTask.getConfiguration();
+ final boolean enableTimestamps = containingTask.getExecutionConfig().areTimestampsEnabled();
+
+ // we read the chained configs, and the order of record writer registrations by output name
+ Map<Integer, StreamConfig> chainedConfigs = configuration.getTransitiveChainedTaskConfigs(userCodeClassloader);
+ chainedConfigs.put(configuration.getVertexID(), configuration);
+
+ // create the final output stream writers
+ // we iterate through all the out edges from this job vertex and create a stream output
+ List<StreamEdge> outEdgesInOrder = configuration.getOutEdgesInOrder(userCodeClassloader);
+ Map<StreamEdge, RecordWriterOutput<?>> streamOutputMap = new HashMap<>(outEdgesInOrder.size());
+ this.streamOutputs = new RecordWriterOutput<?>[outEdgesInOrder.size()];
+
+ for (int i = 0; i < outEdgesInOrder.size(); i++) {
+ StreamEdge outEdge = outEdgesInOrder.get(i);
+
+ RecordWriterOutput<?> streamOutput = createStreamOutput(
+ outEdge, chainedConfigs.get(outEdge.getSourceId()), i,
+ containingTask.getEnvironment(), enableTimestamps, reporter, containingTask.getName());
+
+ streamOutputMap.put(outEdge, streamOutput);
+ this.streamOutputs[i] = streamOutput;
+ }
+
+ // we create the chain of operators and grab the collector that leads into the chain
+ List<StreamOperator<?>> allOps = new ArrayList<>(chainedConfigs.size());
+ this.chainEntryPoint = createOutputCollector(containingTask, configuration,
+ chainedConfigs, userCodeClassloader, streamOutputMap, allOps);
+
+ this.allOperators = allOps.toArray(new StreamOperator<?>[allOps.size() + 1]);
+
+ // add the head operator to the end of the list
+ this.allOperators[this.allOperators.length - 1] = headOperator;
+ }
+
+ //
+
+ public void broadcastCheckpointBarrier(long id, long timestamp) throws IOException, InterruptedException {
+ CheckpointBarrier barrier = new CheckpointBarrier(id, timestamp);
+ for (RecordWriterOutput<?> streamOutput : streamOutputs) {
+ streamOutput.broadcastEvent(barrier);
+ }
+ }
+
+ public RecordWriterOutput<?>[] getStreamOutputs() {
+ return streamOutputs;
+ }
+
+ public StreamOperator<?>[] getAllOperators() {
+ return allOperators;
+ }
+
+ public Output<StreamRecord<OUT>> getChainEntryPoint() {
+ return chainEntryPoint;
+ }
+
+ /**
+ *
+ * This method should be called before finishing the record emission, to make sure any data
+ * that is still buffered will be sent. It also ensures that all data sending related
+ * exceptions are recognized.
+ *
+ * @throws IOException Thrown, if the buffered data cannot be pushed into the output streams.
+ */
+ public void flushOutputs() throws IOException {
+ for (RecordWriterOutput<?> streamOutput : getStreamOutputs()) {
+ streamOutput.flush();
+ }
+ }
+
+ /**
+ * This method releases all resources of the record writer output. It stops the output
+ * flushing thread (if there is one) and releases all buffers currently held by the output
+ * serializers.
+ *
+ * <p>This method should never fail.
+ */
+ public void releaseOutputs() {
+ try {
+ for (RecordWriterOutput<?> streamOutput : streamOutputs) {
+ streamOutput.close();
+ }
+ }
+ finally {
+ // make sure that we release the buffers in any case
+ for (RecordWriterOutput<?> output : streamOutputs) {
+ output.clearBuffers();
+ }
+ }
+ }
+
+ // ------------------------------------------------------------------------
+ // initialization utilities
+ // ------------------------------------------------------------------------
+
+ private static <T> Output<StreamRecord<T>> createOutputCollector(
+ StreamTask<?, ?> containingTask,
+ StreamConfig operatorConfig,
+ Map<Integer, StreamConfig> chainedConfigs,
+ ClassLoader userCodeClassloader,
+ Map<StreamEdge, RecordWriterOutput<?>> streamOutputs,
+ List<StreamOperator<?>> allOperators)
+ {
+ // We create a wrapper that will encapsulate the chained operators and network outputs
+ OutputSelectorWrapper<T> outputSelectorWrapper = operatorConfig.getOutputSelectorWrapper(userCodeClassloader);
+ CollectorWrapper<T> wrapper = new CollectorWrapper<T>(outputSelectorWrapper);
+
+ // create collectors for the network outputs
+ for (StreamEdge outputEdge : operatorConfig.getNonChainedOutputs(userCodeClassloader)) {
+ @SuppressWarnings("unchecked")
+ RecordWriterOutput<T> output = (RecordWriterOutput<T>) streamOutputs.get(outputEdge);
+ wrapper.addCollector(output, outputEdge);
+ }
+
+ // Create collectors for the chained outputs
+ for (StreamEdge outputEdge : operatorConfig.getChainedOutputs(userCodeClassloader)) {
+ int outputId = outputEdge.getTargetId();
+ StreamConfig chainedOpConfig = chainedConfigs.get(outputId);
+
+ Output<StreamRecord<T>> output = createChainedOperator(
+ containingTask, chainedOpConfig, chainedConfigs, userCodeClassloader, streamOutputs, allOperators);
+ wrapper.addCollector(output, outputEdge);
+ }
+ return wrapper;
+ }
+
+ private static <IN, OUT> Output<StreamRecord<IN>> createChainedOperator(
+ StreamTask<?, ?> containingTask,
+ StreamConfig operatorConfig,
+ Map<Integer, StreamConfig> chainedConfigs,
+ ClassLoader userCodeClassloader,
+ Map<StreamEdge, RecordWriterOutput<?>> streamOutputs,
+ List<StreamOperator<?>> allOperators)
+ {
+ // create the output that the operator writes to first. this may recursively create more operators
+ Output<StreamRecord<OUT>> output = createOutputCollector(
+ containingTask, operatorConfig, chainedConfigs, userCodeClassloader, streamOutputs, allOperators);
+
+ // now create the operator and give it the output collector to write its output to
+ OneInputStreamOperator<IN, OUT> chainedOperator = operatorConfig.getStreamOperator(userCodeClassloader);
+ chainedOperator.setup(containingTask, operatorConfig, output);
+
+ allOperators.add(chainedOperator);
+
+ if (containingTask.getExecutionConfig().isObjectReuseEnabled() || chainedOperator.isInputCopyingDisabled()) {
+ return new ChainingOutput<IN>(chainedOperator);
+ }
+ else {
+ TypeSerializer<IN> inSerializer = operatorConfig.getTypeSerializerIn1(userCodeClassloader);
+ return new CopyingChainingOutput<IN>(chainedOperator, inSerializer);
+ }
+ }
+
+ private static <T> RecordWriterOutput<T> createStreamOutput(
+ StreamEdge edge, StreamConfig upStreamConfig, int outputIndex,
+ Environment taskEnvironment, boolean withTimestamps,
+ AccumulatorRegistry.Reporter reporter, String taskName)
+ {
+ TypeSerializer<T> outSerializer = upStreamConfig.getTypeSerializerOut(taskEnvironment.getUserClassLoader());
+
+ @SuppressWarnings("unchecked")
+ StreamPartitioner<T> outputPartitioner = (StreamPartitioner<T>) edge.getPartitioner();
+
+ LOG.debug("Using partitioner {} for output {} of task ", outputPartitioner, outputIndex, taskName);
+
+ ResultPartitionWriter bufferWriter = taskEnvironment.getWriter(outputIndex);
+
+ StreamRecordWriter<SerializationDelegate<StreamRecord<T>>> output =
+ new StreamRecordWriter<>(bufferWriter, outputPartitioner, upStreamConfig.getBufferTimeout());
+ output.setReporter(reporter);
+
+ return new RecordWriterOutput<T>(output, outSerializer, withTimestamps);
+ }
+
+ // ------------------------------------------------------------------------
+ // Collectors for output chaining
+ // ------------------------------------------------------------------------
+
+ private static class ChainingOutput<T> implements Output<StreamRecord<T>> {
+
+ protected final OneInputStreamOperator<T, ?> operator;
+
+ public ChainingOutput(OneInputStreamOperator<T, ?> operator) {
+ this.operator = operator;
+ }
+
+ @Override
+ public void collect(StreamRecord<T> record) {
+ try {
+ operator.setKeyContextElement(record);
+ operator.processElement(record);
+ }
+ catch (Exception e) {
+ throw new ExceptionInChainedOperatorException(e);
+ }
+ }
+
+ @Override
+ public void emitWatermark(Watermark mark) {
+ try {
+ operator.processWatermark(mark);
+ }
+ catch (Exception e) {
+ throw new ExceptionInChainedOperatorException(e);
+ }
+ }
+
+ @Override
+ public void close() {
+ try {
+ operator.close();
+ }
+ catch (Exception e) {
+ throw new ExceptionInChainedOperatorException(e);
+ }
+ }
+ }
+
+ private static class CopyingChainingOutput<T> extends ChainingOutput<T> {
+
+ private final TypeSerializer<T> serializer;
+
+ private final StreamRecord<T> copyRecord;
+
+ public CopyingChainingOutput(OneInputStreamOperator<T, ?> operator, TypeSerializer<T> serializer) {
+ super(operator);
+ this.serializer = serializer;
+ this.copyRecord = new StreamRecord<T>(null, 0L);
+ }
+
+ @Override
+ public void collect(StreamRecord<T> record) {
+ try {
+ T copy = serializer.copy(record.getValue());
+ copyRecord.replace(copy, record.getTimestamp());
+
+ operator.setKeyContextElement(copyRecord);
+ operator.processElement(copyRecord);
+ }
+ catch (Exception e) {
+ throw new RuntimeException("Could not forward element to next operator", e);
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OutputHandler.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OutputHandler.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OutputHandler.java
deleted file mode 100644
index ce659fc..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OutputHandler.java
+++ /dev/null
@@ -1,336 +0,0 @@
-/*
- * 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.runtime.tasks;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.flink.api.common.accumulators.Accumulator;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
-import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
-import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter;
-import org.apache.flink.runtime.plugable.SerializationDelegate;
-import org.apache.flink.streaming.api.watermark.Watermark;
-import org.apache.flink.streaming.runtime.io.CollectorWrapper;
-import org.apache.flink.streaming.runtime.io.RecordWriterOutput;
-import org.apache.flink.streaming.api.collector.selector.OutputSelectorWrapper;
-import org.apache.flink.streaming.api.graph.StreamConfig;
-import org.apache.flink.streaming.api.graph.StreamEdge;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.streaming.api.operators.Output;
-import org.apache.flink.streaming.api.operators.StreamOperator;
-import org.apache.flink.streaming.runtime.io.StreamRecordWriter;
-import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
-import org.apache.flink.streaming.runtime.streamrecord.MultiplexingStreamRecordSerializer;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecordSerializer;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class OutputHandler<OUT> {
-
- private static final Logger LOG = LoggerFactory.getLogger(OutputHandler.class);
-
- private final StreamTask<OUT, ?> vertex;
-
- /** The classloader used to access all user code */
- private final ClassLoader userCodeClassloader;
-
-
- private final Output<StreamRecord<OUT>> outerOutput;
-
- public final List<StreamOperator<?>> chainedOperators;
-
- private final Map<StreamEdge, RecordWriterOutput<?>> outputMap;
-
- private final Map<Integer, StreamConfig> chainedConfigs;
-
- /** Counters for the number of records emitted and bytes written. */
- protected final AccumulatorRegistry.Reporter reporter;
-
-
- public OutputHandler(StreamTask<OUT, ?> vertex, Map<String, Accumulator<?,?>> accumulatorMap,
- AccumulatorRegistry.Reporter reporter) {
-
- // Initialize some fields
- this.vertex = vertex;
- StreamConfig configuration = new StreamConfig(vertex.getTaskConfiguration());
- this.chainedOperators = new ArrayList<StreamOperator<?>>();
- this.outputMap = new HashMap<StreamEdge, RecordWriterOutput<?>>();
- this.userCodeClassloader = vertex.getUserCodeClassLoader();
-
- // We read the chained configs, and the order of record writer
- // registrations by output name
- this.chainedConfigs = configuration.getTransitiveChainedTaskConfigs(userCodeClassloader);
- this.chainedConfigs.put(configuration.getVertexID(), configuration);
-
- List<StreamEdge> outEdgesInOrder = configuration.getOutEdgesInOrder(userCodeClassloader);
-
- this.reporter = reporter;
-
- // We iterate through all the out edges from this job vertex and create
- // a stream output
- for (StreamEdge outEdge : outEdgesInOrder) {
- RecordWriterOutput<?> streamOutput = createStreamOutput(
- outEdge,
- outEdge.getTargetId(),
- chainedConfigs.get(outEdge.getSourceId()),
- outEdgesInOrder.indexOf(outEdge),
- reporter);
- outputMap.put(outEdge, streamOutput);
- }
-
- // We create the outer output that will be passed to the first task
- // in the chain
- this.outerOutput = createChainedCollector(configuration, accumulatorMap);
-
- // Add the head operator to the end of the list
- this.chainedOperators.add(vertex.streamOperator);
- }
-
- public void broadcastBarrier(long id, long timestamp) throws IOException, InterruptedException {
- CheckpointBarrier barrier = new CheckpointBarrier(id, timestamp);
- for (RecordWriterOutput<?> streamOutput : outputMap.values()) {
- streamOutput.broadcastEvent(barrier);
- }
- }
-
- public Collection<RecordWriterOutput<?>> getOutputs() {
- return outputMap.values();
- }
-
- public List<StreamOperator<?>> getChainedOperators(){
- return chainedOperators;
- }
-
- /**
- * This method builds up a nested output which encapsulates all the
- * chained operators and their network output. The result of this recursive
- * call will be passed as output to the first operator in the chain.
- *
- * @param chainedTaskConfig
- * The configuration of the starting operator of the chain, we
- * use this paramater to recursively build the whole chain
- * @return Returns the output for the chain starting from the given
- * config
- */
- @SuppressWarnings("unchecked")
- private <X> Output<StreamRecord<X>> createChainedCollector(StreamConfig chainedTaskConfig, Map<String, Accumulator<?,?>> accumulatorMap) {
-
- // We create a wrapper that will encapsulate the chained operators and
- // network outputs
-
- OutputSelectorWrapper<?> outputSelectorWrapper = chainedTaskConfig.getOutputSelectorWrapper(userCodeClassloader);
- CollectorWrapper wrapper = new CollectorWrapper(outputSelectorWrapper);
-
- // Create collectors for the network outputs
- for (StreamEdge outputEdge : chainedTaskConfig.getNonChainedOutputs(userCodeClassloader)) {
- Output<?> output = outputMap.get(outputEdge);
-
- wrapper.addCollector(output, outputEdge);
- }
-
- // Create collectors for the chained outputs
- for (StreamEdge outputEdge : chainedTaskConfig.getChainedOutputs(userCodeClassloader)) {
- Integer outputId = outputEdge.getTargetId();
-
- Output<?> output = createChainedCollector(chainedConfigs.get(outputId), accumulatorMap);
-
- wrapper.addCollector(output, outputEdge);
- }
-
- if (chainedTaskConfig.isChainStart()) {
- // The current task is the first chained task at this vertex so we
- // return the wrapper
- return (Output<StreamRecord<X>>) wrapper;
- }
- else {
- // The current task is a part of the chain so we get the chainable
- // operator which will be returned and set it up using the wrapper
- OneInputStreamOperator chainableOperator =
- chainedTaskConfig.getStreamOperator(userCodeClassloader);
-
- StreamingRuntimeContext chainedContext = vertex.createRuntimeContext(chainedTaskConfig, accumulatorMap);
- vertex.contexts.add(chainedContext);
-
- chainableOperator.setup(wrapper, chainedContext);
-
- chainedOperators.add(chainableOperator);
- if (vertex.getExecutionConfig().isObjectReuseEnabled() || chainableOperator.isInputCopyingDisabled()) {
- return new ChainingOutput<X>(chainableOperator);
- }
- else {
- TypeSerializer<X> typeSer = chainedTaskConfig.getTypeSerializerIn1(userCodeClassloader);
- TypeSerializer<StreamRecord<X>> inSerializer;
-
- if (vertex.getExecutionConfig().areTimestampsEnabled()) {
- inSerializer = (TypeSerializer<StreamRecord<X>>)
- (TypeSerializer<?>) new MultiplexingStreamRecordSerializer<X>(typeSer);
- }
- else {
- inSerializer = new StreamRecordSerializer<X>(typeSer);
- }
-
- return new CopyingChainingOutput<X>(chainableOperator, inSerializer);
- }
- }
-
- }
-
- public Output<StreamRecord<OUT>> getOutput() {
- return outerOutput;
- }
-
- /**
- * We create the StreamOutput for the specific output given by the id, and
- * the configuration of its source task
- *
- * @param outputVertex
- * Name of the output to which the stream output will be set up
- * @param upStreamConfig
- * The config of upStream task
- * @return The created StreamOutput
- */
- private <T> RecordWriterOutput<T> createStreamOutput(StreamEdge edge, Integer outputVertex,
- StreamConfig upStreamConfig, int outputIndex, AccumulatorRegistry.Reporter reporter) {
-
- TypeSerializer<T> outSerializer = upStreamConfig.getTypeSerializerOut1(vertex.userClassLoader);
-
- @SuppressWarnings("unchecked")
- StreamPartitioner<T> outputPartitioner = (StreamPartitioner<T>) edge.getPartitioner();
-
- ResultPartitionWriter bufferWriter = vertex.getEnvironment().getWriter(outputIndex);
-
- StreamRecordWriter<SerializationDelegate<StreamRecord<T>>> output =
- new StreamRecordWriter<>(bufferWriter, outputPartitioner, upStreamConfig.getBufferTimeout());
-
- output.setReporter(reporter);
-
- RecordWriterOutput<T> streamOutput =
- new RecordWriterOutput<T>(output, outSerializer, vertex.getExecutionConfig().areTimestampsEnabled());
-
- if (LOG.isDebugEnabled()) {
- LOG.debug("Partitioner set: {} with {} outputs for {}", outputPartitioner.getClass()
- .getSimpleName(), outputIndex, vertex.getClass().getSimpleName());
- }
-
- return streamOutput;
- }
-
- /**
- *
- * This method should be called before finishing the record emission, to make sure any data
- * that is still buffered will be sent. It also ensures that all data sending related
- * exceptions are recognized.
- *
- * @throws IOException Thrown, if the buffered data cannot be pushed into the output streams.
- */
- public void flushOutputs() throws IOException {
- for (RecordWriterOutput<?> streamOutput : getOutputs()) {
- streamOutput.flush();
- }
- }
-
- /**
- * This method releases all resources of the record writer output. It stops the output
- * flushing thread (if there is one) and releases all buffers currently held by the output
- * serializers.
- *
- * This method should never fail.
- */
- public void releaseOutputs() {
- try {
- for (RecordWriterOutput<?> streamOutput : getOutputs()) {
- streamOutput.close();
- }
- }
- finally {
- // make sure that we release the buffers in any case
- for (RecordWriterOutput<?> output : getOutputs()) {
- output.clearBuffers();
- }
- }
- }
-
- private static class ChainingOutput<T> implements Output<StreamRecord<T>> {
-
- protected final OneInputStreamOperator<T, ?> operator;
-
- public ChainingOutput(OneInputStreamOperator<T, ?> operator) {
- this.operator = operator;
- }
-
- @Override
- public void collect(StreamRecord<T> record) {
- try {
- operator.getRuntimeContext().setNextInput(record);
- operator.processElement(record);
- }
- catch (Exception e) {
- throw new RuntimeException("Could not forward element to next operator", e);
- }
- }
-
- @Override
- public void emitWatermark(Watermark mark) {
- try {
- operator.processWatermark(mark);
- }
- catch (Exception e) {
- throw new RuntimeException("Could not forward watermark to next operator", e);
- }
- }
-
- @Override
- public void close() {
- try {
- operator.close();
- }
- catch (Exception e) {
- throw new RuntimeException("Could not close() call to next operator", e);
- }
- }
- }
-
- private static class CopyingChainingOutput<T> extends ChainingOutput<T> {
- private final TypeSerializer<StreamRecord<T>> serializer;
-
- public CopyingChainingOutput(OneInputStreamOperator<T, ?> operator,
- TypeSerializer<StreamRecord<T>> serializer) {
- super(operator);
- this.serializer = serializer;
- }
-
- @Override
- public void collect(StreamRecord<T> record) {
- try {
- operator.getRuntimeContext().setNextInput(record);
- operator.processElement(serializer.copy(record));
- }
- catch (Exception e) {
- throw new RuntimeException("Could not forward element to next operator", e);
- }
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java
index fc221f8..3d82275 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/SourceStreamTask.java
@@ -51,15 +51,13 @@ public class SourceStreamTask<OUT> extends StreamTask<OUT, StreamSource<OUT>> {
@Override
protected void run() throws Exception {
final Object checkpointLock = getCheckpointLock();
-
- final SourceOutput<StreamRecord<OUT>> output = new SourceOutput<>(outputHandler.getOutput(), checkpointLock);
-
- streamOperator.run(checkpointLock, output);
+ final SourceOutput<StreamRecord<OUT>> output = new SourceOutput<>(getHeadOutput(), checkpointLock);
+ headOperator.run(checkpointLock, output);
}
@Override
protected void cancelTask() throws Exception {
- streamOperator.cancel();
+ headOperator.cancel();
}
// ------------------------------------------------------------------------
@@ -95,9 +93,7 @@ public class SourceStreamTask<OUT> extends StreamTask<OUT, StreamSource<OUT>> {
@Override
public void collect(T record) {
synchronized (lockObject) {
- if (timerException != null) {
- throw timerException;
- }
+ checkTimerException();
output.collect(record);
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java
index c937e51..2125df1 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationHead.java
@@ -17,7 +17,6 @@
package org.apache.flink.streaming.runtime.tasks;
-import java.util.Collection;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.TimeUnit;
@@ -41,7 +40,7 @@ public class StreamIterationHead<OUT> extends OneInputStreamTask<OUT, OUT> {
@Override
protected void run() throws Exception {
- final String iterationId = configuration.getIterationId();
+ final String iterationId = getConfiguration().getIterationId();
if (iterationId == null || iterationId.length() == 0) {
throw new Exception("Missing iteration ID in the task configuration");
}
@@ -49,7 +48,7 @@ public class StreamIterationHead<OUT> extends OneInputStreamTask<OUT, OUT> {
final String brokerID = createBrokerIdString(getEnvironment().getJobID(), iterationId ,
getEnvironment().getIndexInSubtaskGroup());
- final long iterationWaitTime = configuration.getIterationWaitTime();
+ final long iterationWaitTime = getConfiguration().getIterationWaitTime();
final boolean shouldWait = iterationWaitTime > 0;
final BlockingQueue<StreamRecord<OUT>> dataChannel = new ArrayBlockingQueue<StreamRecord<OUT>>(1);
@@ -61,8 +60,7 @@ public class StreamIterationHead<OUT> extends OneInputStreamTask<OUT, OUT> {
// do the work
try {
@SuppressWarnings("unchecked")
- Collection<RecordWriterOutput<OUT>> outputs =
- (Collection<RecordWriterOutput<OUT>>) (Collection<?>) outputHandler.getOutputs();
+ RecordWriterOutput<OUT>[] outputs = (RecordWriterOutput<OUT>[]) getStreamOutputs();
// If timestamps are enabled we make sure to remove cyclic watermark dependencies
if (getExecutionConfig().areTimestampsEnabled()) {
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationTail.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationTail.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationTail.java
index fdce52d..9bb5311 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationTail.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamIterationTail.java
@@ -36,7 +36,7 @@ public class StreamIterationTail<IN> extends OneInputStreamTask<IN, IN> {
public void init() throws Exception {
super.init();
- final String iterationId = configuration.getIterationId();
+ final String iterationId = getConfiguration().getIterationId();
if (iterationId == null || iterationId.length() == 0) {
throw new Exception("Missing iteration ID in the task configuration");
}
@@ -44,7 +44,7 @@ public class StreamIterationTail<IN> extends OneInputStreamTask<IN, IN> {
final String brokerID = StreamIterationHead.createBrokerIdString(getEnvironment().getJobID(), iterationId,
getEnvironment().getIndexInSubtaskGroup());
- final long iterationWaitTime = configuration.getIterationWaitTime();
+ final long iterationWaitTime = getConfiguration().getIterationWaitTime();
LOG.info("Iteration tail {} trying to acquire feedback queue under {}", getName(), brokerID);
@@ -54,7 +54,7 @@ public class StreamIterationTail<IN> extends OneInputStreamTask<IN, IN> {
LOG.info("Iteration tail {} acquired feedback queue {}", getName(), brokerID);
- this.streamOperator = new RecordPusher<>(dataChannel, iterationWaitTime);
+ this.headOperator = new RecordPusher<>(dataChannel, iterationWaitTime);
}
private static class RecordPusher<IN> extends AbstractStreamOperator<IN> implements OneInputStreamOperator<IN, IN> {
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
index 16b8f55..bbfd233 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
@@ -17,121 +17,125 @@
package org.apache.flink.streaming.runtime.tasks;
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
import java.util.Map;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.TimeUnit;
-import org.apache.commons.collections.CollectionUtils;
-import org.apache.commons.collections.functors.NotNullPredicate;
-
import org.apache.flink.api.common.accumulators.Accumulator;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.IllegalConfigurationException;
import org.apache.flink.runtime.accumulators.AccumulatorRegistry;
import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
import org.apache.flink.runtime.jobgraph.tasks.StatefulTask;
-import org.apache.flink.runtime.state.FileStateHandle;
-import org.apache.flink.runtime.state.LocalStateHandle;
-import org.apache.flink.runtime.state.StateHandle;
-import org.apache.flink.runtime.state.StateHandleProvider;
import org.apache.flink.runtime.taskmanager.DispatcherThreadFactory;
import org.apache.flink.runtime.util.event.EventListener;
import org.apache.flink.streaming.api.graph.StreamConfig;
-import org.apache.flink.streaming.api.operators.StatefulStreamOperator;
+import org.apache.flink.streaming.api.operators.Output;
import org.apache.flink.streaming.api.operators.StreamOperator;
-import org.apache.flink.streaming.api.state.OperatorStateHandle;
-import org.apache.flink.streaming.api.state.WrapperStateHandle;
-
+import org.apache.flink.streaming.api.state.StateBackend;
+import org.apache.flink.streaming.api.state.StateBackendFactory;
+import org.apache.flink.streaming.api.state.filesystem.FsStateBackend;
+import org.apache.flink.streaming.api.state.filesystem.FsStateBackendFactory;
+import org.apache.flink.streaming.api.state.memory.MemoryStateBackend;
+import org.apache.flink.streaming.runtime.io.RecordWriterOutput;
import org.apache.flink.streaming.runtime.operators.Triggerable;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-
/**
- * Base Invokable for all {@code StreamTasks}. A {@code StreamTask} processes input and forwards
- * elements and watermarks to a {@link StreamOperator}.
+ * Base class for all streaming tasks. A task is the unit of local processing that is deployed
+ * and executed by the TaskManagers. Each task runs one or more {@link StreamOperator}s which form
+ * the Task's operator chain. Operators that are chained together execute synchronously in the
+ * same thread and hence on the same stream partition. A common case for these chaines
+ * are successive map/flatmap/filter tasks.
+ *
+ * <p>The task chain contains one "head" operator and multiple chained operators.
+ * The StreamTask is specialized for the type of the head operator: one-input and two-input tasks,
+ * as well as for sources, iteration heads and iteration tails.
+ *
+ * <p>The Task class deals with the setup of the streams read by the head operator, and the streams
+ * produced by the operators at the ends of the operator chain. Note that the chain may fork and
+ * thus have multiple ends.
*
+ * The life cycle of the task is set up as follows:
* <pre>
- *
* -- registerInputOutput()
* |
- * +----> Create basic utils (config, etc) and load operators
- * +----> operator specific init()
+ * +----> Create basic utils (config, etc) and load the chain of operators
+ * +----> operators.setup()
+ * +----> task specific init()
*
- * -- restoreState()
+ * -- restoreState() -> restores state of all operators in the chain
*
* -- invoke()
* |
- * +----> open operators()
+ * +----> open-operators()
* +----> run()
- * +----> close operators()
+ * +----> close-operators()
+ * +----> dispose-operators()
* +----> common cleanup
- * +----> operator specific cleanup()
+ * +----> task specific cleanup()
* </pre>
*
- * <p>
- * {@code StreamTask} has a lock object called {@code lock}. All calls to methods on a
+ * <p> The {@code StreamTask} has a lock object called {@code lock}. All calls to methods on a
* {@code StreamOperator} must be synchronized on this lock object to ensure that no methods
* are called concurrently.
*
* @param <OUT>
- * @param <O>
+ * @param <Operator>
*/
-public abstract class StreamTask<OUT, O extends StreamOperator<OUT>> extends AbstractInvokable implements StatefulTask<StateHandle<Serializable>> {
+public abstract class StreamTask<OUT, Operator extends StreamOperator<OUT>>
+ extends AbstractInvokable
+ implements StatefulTask<StreamTaskStateList> {
/** The thread group that holds all trigger timer threads */
public static final ThreadGroup TRIGGER_THREAD_GROUP = new ThreadGroup("Triggers");
- private static final Logger LOG = LoggerFactory.getLogger(StreamTask.class);
-
+ /** The logger used by the StreamTask and its subclasses */
+ protected static final Logger LOG = LoggerFactory.getLogger(StreamTask.class);
+
+ // ------------------------------------------------------------------------
+
/**
* All interaction with the {@code StreamOperator} must be synchronized on this lock object to ensure that
- * we don't have concurrent method calls.
+ * we don't have concurrent method calls that void consistent checkpoints.
*/
- protected final Object lock = new Object();
-
- private final EventListener<CheckpointBarrier> checkpointBarrierListener;
+ private final Object lock = new Object();
- protected final List<StreamingRuntimeContext> contexts;
+ /** the head operator that consumes the input streams of this task */
+ protected Operator headOperator;
- protected StreamingRuntimeContext headContext;
+ /** The chain of operators executed by this task */
+ private OperatorChain<OUT> operatorChain;
- protected StreamConfig configuration;
-
- protected ClassLoader userClassLoader;
+ /** The configuration of this streaming task */
+ private StreamConfig configuration;
- /** The executor service that */
+ /** The class loader used to load dynamic classes of a job */
+ private ClassLoader userClassLoader;
+
+ /** The state backend that stores the state and checkpoints for this task */
+ private StateBackend<?> stateBackend;
+
+ /** The executor service that schedules and calls the triggers of this task*/
private ScheduledExecutorService timerService;
+
+ /** The map of user-defined accumulators of this task */
+ private Map<String, Accumulator<?, ?>> accumulatorMap;
- /**
- * This field is used to forward an exception that is caught in the timer thread. Subclasses
- * must ensure that exceptions stored here get thrown on the actual execution Thread.
- */
- protected volatile TimerException timerException = null;
-
- protected OutputHandler<OUT> outputHandler;
-
- protected O streamOperator;
-
- protected boolean hasChainedOperators;
-
+ /** This field is used to forward an exception that is caught in the timer thread. Subclasses
+ * must ensure that exceptions stored here get thrown on the actual execution Thread. */
+ private volatile TimerException timerException;
+
/** Flag to mark the task "in operation", in which case check
* needs to be initialized to true, so that early cancel() before invoke() behaves correctly */
private volatile boolean isRunning;
- // ------------------------------------------------------------------------
-
- public StreamTask() {
- checkpointBarrierListener = new CheckpointBarrierListener();
- contexts = new ArrayList<>();
- }
// ------------------------------------------------------------------------
// Life cycle methods for specific implementations
@@ -152,34 +156,27 @@ public abstract class StreamTask<OUT, O extends StreamOperator<OUT>> extends Abs
@Override
public final void registerInputOutput() throws Exception {
LOG.debug("Begin initialization for {}", getName());
+
+ AccumulatorRegistry accumulatorRegistry = getEnvironment().getAccumulatorRegistry();
userClassLoader = getUserCodeClassLoader();
configuration = new StreamConfig(getTaskConfiguration());
-
- streamOperator = configuration.getStreamOperator(userClassLoader);
-
- // Create and register Accumulators
- AccumulatorRegistry accumulatorRegistry = getEnvironment().getAccumulatorRegistry();
- Map<String, Accumulator<?, ?>> accumulatorMap = accumulatorRegistry.getUserMap();
- AccumulatorRegistry.Reporter reporter = accumulatorRegistry.getReadWriteReporter();
-
- outputHandler = new OutputHandler<>(this, accumulatorMap, reporter);
-
- if (streamOperator != null) {
- // IterationHead and IterationTail don't have an Operator...
-
- //Create context of the head operator
- headContext = createRuntimeContext(configuration, accumulatorMap);
- this.contexts.add(headContext);
- streamOperator.setup(outputHandler.getOutput(), headContext);
+ accumulatorMap = accumulatorRegistry.getUserMap();
+
+ stateBackend = createStateBackend();
+ stateBackend.initializeForJob(getEnvironment().getJobID());
+
+ headOperator = configuration.getStreamOperator(userClassLoader);
+ operatorChain = new OperatorChain<>(this, headOperator, accumulatorRegistry.getReadWriteReporter());
+
+ if (headOperator != null) {
+ headOperator.setup(this, configuration, operatorChain.getChainEntryPoint());
}
- hasChainedOperators = outputHandler.getChainedOperators().size() != 1;
-
- this.timerService = Executors.newSingleThreadScheduledExecutor(
+ timerService = Executors.newSingleThreadScheduledExecutor(
new DispatcherThreadFactory(TRIGGER_THREAD_GROUP, "Time Trigger for " + getName()));
- // operator specific initialization
+ // task specific initialization
init();
LOG.debug("Finish initialization for {}", getName());
@@ -211,7 +208,7 @@ public abstract class StreamTask<OUT, O extends StreamOperator<OUT>> extends Abs
closeAllOperators();
// make sure all data is flushed
- outputHandler.flushOutputs();
+ operatorChain.flushOutputs();
// make an attempt to dispose the operators such that failures in the dispose call
// still let the computation fail
@@ -224,8 +221,8 @@ public abstract class StreamTask<OUT, O extends StreamOperator<OUT>> extends Abs
timerService.shutdown();
// release the output resources. this method should never fail.
- if (outputHandler != null) {
- outputHandler.releaseOutputs();
+ if (operatorChain != null) {
+ operatorChain.releaseOutputs();
}
// we must! perform this cleanup
@@ -252,41 +249,33 @@ public abstract class StreamTask<OUT, O extends StreamOperator<OUT>> extends Abs
}
private void openAllOperators() throws Exception {
- for (StreamOperator<?> operator : outputHandler.getChainedOperators()) {
- if (operator != null) {
- operator.open(getTaskConfiguration());
- }
+ for (StreamOperator<?> operator : operatorChain.getAllOperators()) {
+ operator.open();
}
}
private void closeAllOperators() throws Exception {
// We need to close them first to last, since upstream operators in the chain might emit
// elements in their close methods.
- for (int i = outputHandler.getChainedOperators().size() - 1; i >= 0; i--) {
- StreamOperator<?> operator = outputHandler.getChainedOperators().get(i);
- if (operator != null) {
- operator.close();
- }
+ StreamOperator<?>[] allOperators = operatorChain.getAllOperators();
+ for (int i = allOperators.length - 1; i >= 0; i--) {
+ allOperators[i].close();
}
}
private void tryDisposeAllOperators() throws Exception {
- for (StreamOperator<?> operator : outputHandler.getChainedOperators()) {
- if (operator != null) {
- operator.dispose();
- }
+ for (StreamOperator<?> operator : operatorChain.getAllOperators()) {
+ operator.dispose();
}
}
private void disposeAllOperators() {
- for (StreamOperator<?> operator : outputHandler.getChainedOperators()) {
- if (operator != null) {
- try {
- operator.dispose();
- }
- catch (Throwable t) {
- LOG.error("Error during disposal of stream operator.", t);
- }
+ for (StreamOperator<?> operator : operatorChain.getAllOperators()) {
+ try {
+ operator.dispose();
+ }
+ catch (Throwable t) {
+ LOG.error("Error during disposal of stream operator.", t);
}
}
}
@@ -300,8 +289,8 @@ public abstract class StreamTask<OUT, O extends StreamOperator<OUT>> extends Abs
* shutdown is attempted, and cause threads to linger for longer than needed.
*/
@Override
- @SuppressWarnings("FinalizeDoesntCallSuperFinalize")
- protected void finalize() {
+ protected void finalize() throws Throwable {
+ super.finalize();
if (timerService != null) {
if (!timerService.isTerminated()) {
LOG.warn("Timer service was not shut down. Shutting down in finalize().");
@@ -322,73 +311,84 @@ public abstract class StreamTask<OUT, O extends StreamOperator<OUT>> extends Abs
return getEnvironment().getTaskNameWithSubtasks();
}
+ /**
+ * Gets the lock object on which all operations that involve data and state mutation have to lock.
+
+ * @return The checkpoint lock object.
+ */
public Object getCheckpointLock() {
return lock;
}
+
+ public StreamConfig getConfiguration() {
+ return configuration;
+ }
+
+ public Map<String, Accumulator<?, ?>> getAccumulatorMap() {
+ return accumulatorMap;
+ }
+
+ public Output<StreamRecord<OUT>> getHeadOutput() {
+ return operatorChain.getChainEntryPoint();
+ }
+
+ public RecordWriterOutput<?>[] getStreamOutputs() {
+ return operatorChain.getStreamOutputs();
+ }
// ------------------------------------------------------------------------
// Checkpoint and Restore
// ------------------------------------------------------------------------
-
- @SuppressWarnings("unchecked")
+
@Override
- public void setInitialState(StateHandle<Serializable> stateHandle) throws Exception {
-
- // We retrieve end restore the states for the chained operators.
- List<Tuple2<StateHandle<Serializable>, Map<String, OperatorStateHandle>>> chainedStates =
- (List<Tuple2<StateHandle<Serializable>, Map<String, OperatorStateHandle>>>) stateHandle.getState(this.userClassLoader);
-
- // We restore all stateful operators
- for (int i = 0; i < chainedStates.size(); i++) {
- Tuple2<StateHandle<Serializable>, Map<String, OperatorStateHandle>> state = chainedStates.get(i);
- // If state is not null we need to restore it
- if (state != null) {
- StreamOperator<?> chainedOperator = outputHandler.getChainedOperators().get(i);
- ((StatefulStreamOperator<?>) chainedOperator).restoreInitialState(state);
+ public void setInitialState(StreamTaskStateList initialState) throws Exception {
+ LOG.info("Restoring checkpointed state to task {}", getName());
+
+ final StreamOperator<?>[] allOperators = operatorChain.getAllOperators();
+ final StreamTaskState[] states = initialState.getState(userClassLoader);
+
+ for (int i = 0; i < states.length; i++) {
+ StreamTaskState state = states[i];
+ StreamOperator<?> operator = allOperators[i];
+
+ if (state != null && operator != null) {
+ LOG.debug("Task {} in chain ({}) has checkpointed state", i, getName());
+ operator.restoreState(state);
+ }
+ else if (operator != null) {
+ LOG.debug("Task {} in chain ({}) does not have checkpointed state", i, getName());
}
}
}
@Override
public void triggerCheckpoint(long checkpointId, long timestamp) throws Exception {
-
LOG.debug("Starting checkpoint {} on task {}", checkpointId, getName());
synchronized (lock) {
if (isRunning) {
- try {
- // We wrap the states of the chained operators in a list, marking non-stateful operators with null
- List<Tuple2<StateHandle<Serializable>, Map<String, OperatorStateHandle>>> chainedStates = new ArrayList<>();
- // A wrapper handle is created for the List of statehandles
- WrapperStateHandle stateHandle;
- try {
-
- // We construct a list of states for chained tasks
- for (StreamOperator<?> chainedOperator : outputHandler.getChainedOperators()) {
- if (chainedOperator instanceof StatefulStreamOperator) {
- chainedStates.add(((StatefulStreamOperator<?>) chainedOperator)
- .getStateSnapshotFromFunction(checkpointId, timestamp));
- }else{
- chainedStates.add(null);
- }
- }
-
- stateHandle = CollectionUtils.exists(chainedStates,
- NotNullPredicate.INSTANCE) ? new WrapperStateHandle(chainedStates) : null;
- }
- catch (Exception e) {
- throw new Exception("Error while drawing snapshot of the user state.", e);
+ // since both state checkpointing and downstream barrier emission occurs in this
+ // lock scope, they are an atomic operation regardless of the order in which they occur
+ // we immediately emit the checkpoint barriers, so the downstream operators can start
+ // their checkpoint work as soon as possible
+ operatorChain.broadcastCheckpointBarrier(checkpointId, timestamp);
+
+ // now draw the state snapshot
+ try {
+ final StreamOperator<?>[] allOperators = operatorChain.getAllOperators();
+ final StreamTaskState[] states = new StreamTaskState[allOperators.length];
+
+ for (int i = 0; i < states.length; i++) {
+ StreamTaskState state = allOperators[i].snapshotOperatorState(checkpointId, timestamp);
+ states[i] = state.isEmpty() ? null : state;
}
- // now emit the checkpoint barriers
- outputHandler.broadcastBarrier(checkpointId, timestamp);
-
- // now confirm the checkpoint
- if (stateHandle == null) {
+ StreamTaskStateList allStates = new StreamTaskStateList(states);
+ if (allStates.isEmpty()) {
getEnvironment().acknowledgeCheckpoint(checkpointId);
} else {
- getEnvironment().acknowledgeCheckpoint(checkpointId, stateHandle);
+ getEnvironment().acknowledgeCheckpoint(checkpointId, allStates);
}
}
catch (Exception e) {
@@ -404,64 +404,85 @@ public abstract class StreamTask<OUT, O extends StreamOperator<OUT>> extends Abs
public void notifyCheckpointComplete(long checkpointId) throws Exception {
synchronized (lock) {
if (isRunning) {
- for (StreamOperator<?> chainedOperator : outputHandler.getChainedOperators()) {
- if (chainedOperator instanceof StatefulStreamOperator) {
- ((StatefulStreamOperator<?>) chainedOperator).notifyCheckpointComplete(checkpointId);
- }
+ LOG.debug("Notification of complete checkpoint for task {}", getName());
+
+ for (StreamOperator<?> operator : operatorChain.getAllOperators()) {
+ operator.notifyOfCompletedCheckpoint(checkpointId);
}
}
+ else {
+ LOG.debug("Ignoring notification of complete checkpoint for not-running task {}", getName());
+ }
}
}
// ------------------------------------------------------------------------
// State backend
// ------------------------------------------------------------------------
-
- private StateHandleProvider<Serializable> getStateHandleProvider() {
- StateHandleProvider<Serializable> provider = configuration.getStateHandleProvider(userClassLoader);
- // If the user did not specify a provider in the program we try to get it from the config
- if (provider == null) {
+ /**
+ * Gets the state backend used by this task. The state backend defines how to maintain the
+ * key/value state and how and where to store state snapshots.
+ *
+ * @return The state backend used by this task.
+ */
+ public StateBackend<?> getStateBackend() {
+ return stateBackend;
+ }
+
+ private StateBackend<?> createStateBackend() throws Exception {
+ StateBackend<?> configuredBackend = configuration.getStateBackend(userClassLoader);
+
+ if (configuredBackend != null) {
+ // backend has been configured on the environment
+ LOG.info("Using user-defined state backend: " + configuredBackend);
+ return configuredBackend;
+ }
+ else {
+ // see if we have a backend specified in the configuration
Configuration flinkConfig = getEnvironment().getTaskManagerInfo().getConfiguration();
- String backendName = flinkConfig.getString(ConfigConstants.STATE_BACKEND,
- ConfigConstants.DEFAULT_STATE_BACKEND).toUpperCase();
-
- StateBackend backend;
-
- try {
- backend = StateBackend.valueOf(backendName);
- } catch (Exception e) {
- throw new RuntimeException(backendName + " is not a valid state backend.\nSupported backends: jobmanager, filesystem.");
+ String backendName = flinkConfig.getString(ConfigConstants.STATE_BACKEND, null);
+
+ if (backendName == null) {
+ LOG.warn("No state backend has been specified, using default state backend (Memory / JobManager)");
+ backendName = "jobmanager";
}
+
+ backendName = backendName.toLowerCase();
+ switch (backendName) {
+ case "jobmanager":
+ LOG.info("State backend is set to heap memory (checkpoint to jobmanager)");
+ return MemoryStateBackend.defaultInstance();
+
+ case "filesystem":
+ FsStateBackend backend = new FsStateBackendFactory().createFromConfig(flinkConfig);
+ LOG.info("State backend is set to heap memory (checkpoints to filesystem \""
+ + backend.getBasePath() + "\")");
+ return backend;
+
+ default:
+ try {
+ @SuppressWarnings("rawtypes")
+ Class<? extends StateBackendFactory> clazz =
+ Class.forName(backendName, false, userClassLoader).asSubclass(StateBackendFactory.class);
- switch (backend) {
- case JOBMANAGER:
- LOG.info("State backend for state checkpoints is set to jobmanager.");
- return new LocalStateHandle.LocalStateHandleProvider<>();
- case FILESYSTEM:
- String checkpointDir = flinkConfig.getString(ConfigConstants.STATE_BACKEND_FS_DIR, null);
- if (checkpointDir != null) {
- LOG.info("State backend for state checkpoints is set to filesystem with directory: "
- + checkpointDir);
- return FileStateHandle.createProvider(checkpointDir);
- } else {
- throw new RuntimeException(
- "For filesystem checkpointing, a checkpoint directory needs to be specified.\nFor example: \"state.backend.dir: hdfs://checkpoints\"");
+ return (StateBackend<?>) clazz.newInstance();
+ }
+ catch (ClassNotFoundException e) {
+ throw new IllegalConfigurationException("Cannot find configured state backend: " + backendName);
+ }
+ catch (ClassCastException e) {
+ throw new IllegalConfigurationException("The class configured under '" +
+ ConfigConstants.STATE_BACKEND + "' is not a valid state backend factory (" +
+ backendName + ')');
+ }
+ catch (Throwable t) {
+ throw new IllegalConfigurationException("Cannot create configured state backend", t);
}
- default:
- throw new RuntimeException("Backend " + backend + " is not supported yet.");
}
-
- } else {
- LOG.info("Using user defined state backend for streaming checkpoitns.");
- return provider;
}
}
- private enum StateBackend {
- JOBMANAGER, FILESYSTEM
- }
-
/**
* Registers a timer.
*/
@@ -473,41 +494,37 @@ public abstract class StreamTask<OUT, O extends StreamOperator<OUT>> extends Abs
delay,
TimeUnit.MILLISECONDS);
}
+
+ public void checkTimerException() throws TimerException {
+ if (timerException != null) {
+ throw timerException;
+ }
+ }
// ------------------------------------------------------------------------
// Utilities
// ------------------------------------------------------------------------
-
- public StreamingRuntimeContext createRuntimeContext(StreamConfig conf, Map<String, Accumulator<?,?>> accumulatorMap) {
- KeySelector<?,Serializable> statePartitioner = conf.getStatePartitioner(userClassLoader);
-
- return new StreamingRuntimeContext(getEnvironment(), getExecutionConfig(),
- statePartitioner, getStateHandleProvider(), accumulatorMap, this);
- }
@Override
public String toString() {
return getName();
}
- // ------------------------------------------------------------------------
-
- public EventListener<CheckpointBarrier> getCheckpointBarrierListener() {
- return this.checkpointBarrierListener;
- }
-
- private class CheckpointBarrierListener implements EventListener<CheckpointBarrier> {
-
- @Override
- public void onEvent(CheckpointBarrier barrier) {
- try {
- triggerCheckpoint(barrier.getId(), barrier.getTimestamp());
- }
- catch (Exception e) {
- throw new RuntimeException("Error triggering a checkpoint as the result of receiving checkpoint barrier", e);
+ protected final EventListener<CheckpointBarrier> getCheckpointBarrierListener() {
+ return new EventListener<CheckpointBarrier>() {
+ @Override
+ public void onEvent(CheckpointBarrier barrier) {
+ try {
+ triggerCheckpoint(barrier.getId(), barrier.getTimestamp());
+ }
+ catch (Exception e) {
+ throw new RuntimeException("Error triggering a checkpoint as the result of receiving checkpoint barrier", e);
+ }
}
- }
+ };
}
+
+ // ------------------------------------------------------------------------
/**
* Internal task that is invoked by the timer service and triggers the target.
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskState.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskState.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskState.java
new file mode 100644
index 0000000..2fce7af
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTaskState.java
@@ -0,0 +1,108 @@
+/*
+ * 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.runtime.tasks;
+
+import org.apache.flink.runtime.state.StateHandle;
+import org.apache.flink.streaming.api.state.KvStateSnapshot;
+
+import java.io.Serializable;
+
+/**
+ * The state checkpointed by a {@link org.apache.flink.streaming.api.operators.AbstractStreamOperator}.
+ * This state consists of any combination of those three:
+ * <ul>
+ * <li>The state of the stream operator, if it implements the Checkpointed interface.</li>
+ * <li>The state of the user function, if it implements the Checkpointed interface.</li>
+ * <li>The key/value state of the operator, if it executes on a KeyedDataStream.</li>
+ * </ul>
+ */
+public class StreamTaskState implements Serializable {
+
+ private static final long serialVersionUID = 1L;
+
+ private StateHandle<?> operatorState;
+
+ private StateHandle<Serializable> functionState;
+
+ private KvStateSnapshot<?, ?, ?> kvState;
+
+ // ------------------------------------------------------------------------
+
+ public StateHandle<?> getOperatorState() {
+ return operatorState;
+ }
+
+ public void setOperatorState(StateHandle<?> operatorState) {
+ this.operatorState = operatorState;
+ }
+
+ public StateHandle<Serializable> getFunctionState() {
+ return functionState;
+ }
+
+ public void setFunctionState(StateHandle<Serializable> functionState) {
+ this.functionState = functionState;
+ }
+
+ public KvStateSnapshot<?, ?, ?> getKvState() {
+ return kvState;
+ }
+
+ public void setKvState(KvStateSnapshot<?, ?, ?> kvState) {
+ this.kvState = kvState;
+ }
+
+ // ------------------------------------------------------------------------
+
+ /**
+ * Checks if this state object actually contains any state, or if all of the state
+ * fields are null.
+ *
+ * @return True, if all state is null, false if at least one state is not null.
+ */
+ public boolean isEmpty() {
+ return operatorState == null & functionState == null & kvState == null;
+ }
+
+ /**
+ * Discards all the contained states and sets them to null.
+ *
+ * @throws Exception Forwards exceptions that occur when releasing the
+ * state handles and snapshots.
+ */
+ public void discardState() throws Exception {
+ StateHandle<?> operatorState = this.operatorState;
+ StateHandle<?> functionState = this.functionState;
+ KvStateSnapshot<?, ?, ?> kvState = this.kvState;
+
+ if (operatorState != null) {
+ operatorState.discardState();
+ }
+ if (functionState != null) {
+ functionState.discardState();
+ }
+ if (kvState != null) {
+ kvState.discardState();
+ }
+
+ this.operatorState = null;
+ this.functionState = null;
+ this.kvState = null;
+ }
+}
[15/24] flink git commit: [hotfix] StreamTask and OperatorChain
properly clean up partially initialized resources upon failures during
initialization
Posted by se...@apache.org.
[hotfix] StreamTask and OperatorChain properly clean up partially initialized resources upon failures during initialization
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/d5a016cf
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/d5a016cf
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/d5a016cf
Branch: refs/heads/master
Commit: d5a016cff5c71dc6a6b995ebd6eb5757131f292a
Parents: f2d5038
Author: Stephan Ewen <se...@apache.org>
Authored: Fri Oct 9 18:21:35 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Fri Oct 16 15:26:10 2015 +0200
----------------------------------------------------------------------
.../streaming/runtime/tasks/OperatorChain.java | 56 ++++---
.../streaming/runtime/tasks/StreamTask.java | 149 ++++++++++++-------
2 files changed, 135 insertions(+), 70 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/d5a016cf/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java
index 9df3a5d..b42b888 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java
@@ -74,29 +74,47 @@ public class OperatorChain<OUT> {
Map<StreamEdge, RecordWriterOutput<?>> streamOutputMap = new HashMap<>(outEdgesInOrder.size());
this.streamOutputs = new RecordWriterOutput<?>[outEdgesInOrder.size()];
- for (int i = 0; i < outEdgesInOrder.size(); i++) {
- StreamEdge outEdge = outEdgesInOrder.get(i);
+ // from here on, we need to make sure that the output writers are shut down again on failure
+ boolean success = false;
+ try {
+ for (int i = 0; i < outEdgesInOrder.size(); i++) {
+ StreamEdge outEdge = outEdgesInOrder.get(i);
+
+ RecordWriterOutput<?> streamOutput = createStreamOutput(
+ outEdge, chainedConfigs.get(outEdge.getSourceId()), i,
+ containingTask.getEnvironment(), enableTimestamps, reporter, containingTask.getName());
+
+ this.streamOutputs[i] = streamOutput;
+ streamOutputMap.put(outEdge, streamOutput);
+ }
+
+ // we create the chain of operators and grab the collector that leads into the chain
+ List<StreamOperator<?>> allOps = new ArrayList<>(chainedConfigs.size());
+ this.chainEntryPoint = createOutputCollector(containingTask, configuration,
+ chainedConfigs, userCodeClassloader, streamOutputMap, allOps);
- RecordWriterOutput<?> streamOutput = createStreamOutput(
- outEdge, chainedConfigs.get(outEdge.getSourceId()), i,
- containingTask.getEnvironment(), enableTimestamps, reporter, containingTask.getName());
-
- streamOutputMap.put(outEdge, streamOutput);
- this.streamOutputs[i] = streamOutput;
+ this.allOperators = allOps.toArray(new StreamOperator<?>[allOps.size() + 1]);
+
+ // add the head operator to the end of the list
+ this.allOperators[this.allOperators.length - 1] = headOperator;
+
+ success = true;
+ }
+ finally {
+ // make sure we clean up after ourselves in case of a failure after acquiring
+ // the first resources
+ if (!success) {
+ for (RecordWriterOutput<?> output : this.streamOutputs) {
+ if (output != null) {
+ output.close();
+ output.clearBuffers();
+ }
+ }
+ }
}
-
- // we create the chain of operators and grab the collector that leads into the chain
- List<StreamOperator<?>> allOps = new ArrayList<>(chainedConfigs.size());
- this.chainEntryPoint = createOutputCollector(containingTask, configuration,
- chainedConfigs, userCodeClassloader, streamOutputMap, allOps);
-
- this.allOperators = allOps.toArray(new StreamOperator<?>[allOps.size() + 1]);
- // add the head operator to the end of the list
- this.allOperators[this.allOperators.length - 1] = headOperator;
}
-
- //
+
public void broadcastCheckpointBarrier(long id, long timestamp) throws IOException, InterruptedException {
CheckpointBarrier barrier = new CheckpointBarrier(id, timestamp);
http://git-wip-us.apache.org/repos/asf/flink/blob/d5a016cf/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
index 5bf7d8e..b53d9c4 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
@@ -127,6 +127,9 @@ public abstract class StreamTask<OUT, Operator extends StreamOperator<OUT>>
/** The map of user-defined accumulators of this task */
private Map<String, Accumulator<?, ?>> accumulatorMap;
+
+ /** The state to be restored once the initialization is done */
+ private StreamTaskStateList lazyRestoreState;
/** This field is used to forward an exception that is caught in the timer thread. Subclasses
* must ensure that exceptions stored here get thrown on the actual execution Thread. */
@@ -155,31 +158,44 @@ public abstract class StreamTask<OUT, Operator extends StreamOperator<OUT>>
@Override
public final void registerInputOutput() throws Exception {
- LOG.debug("Begin initialization for {}", getName());
+ LOG.debug("registerInputOutput for {}", getName());
- AccumulatorRegistry accumulatorRegistry = getEnvironment().getAccumulatorRegistry();
-
- userClassLoader = getUserCodeClassLoader();
- configuration = new StreamConfig(getTaskConfiguration());
- accumulatorMap = accumulatorRegistry.getUserMap();
-
- stateBackend = createStateBackend();
- stateBackend.initializeForJob(getEnvironment().getJobID());
-
- headOperator = configuration.getStreamOperator(userClassLoader);
- operatorChain = new OperatorChain<>(this, headOperator, accumulatorRegistry.getReadWriteReporter());
-
- if (headOperator != null) {
- headOperator.setup(this, configuration, operatorChain.getChainEntryPoint());
- }
+ boolean initializationCompleted = false;
+ try {
+ AccumulatorRegistry accumulatorRegistry = getEnvironment().getAccumulatorRegistry();
- timerService = Executors.newSingleThreadScheduledExecutor(
- new DispatcherThreadFactory(TRIGGER_THREAD_GROUP, "Time Trigger for " + getName()));
+ userClassLoader = getUserCodeClassLoader();
+ configuration = new StreamConfig(getTaskConfiguration());
+ accumulatorMap = accumulatorRegistry.getUserMap();
- // task specific initialization
- init();
-
- LOG.debug("Finish initialization for {}", getName());
+ stateBackend = createStateBackend();
+ stateBackend.initializeForJob(getEnvironment().getJobID());
+
+ headOperator = configuration.getStreamOperator(userClassLoader);
+ operatorChain = new OperatorChain<>(this, headOperator, accumulatorRegistry.getReadWriteReporter());
+
+ if (headOperator != null) {
+ headOperator.setup(this, configuration, operatorChain.getChainEntryPoint());
+ }
+
+ timerService = Executors.newSingleThreadScheduledExecutor(
+ new DispatcherThreadFactory(TRIGGER_THREAD_GROUP, "Time Trigger for " + getName()));
+
+ // task specific initialization
+ init();
+
+ initializationCompleted = true;
+ }
+ finally {
+ if (!initializationCompleted) {
+ if (timerService != null) {
+ timerService.shutdownNow();
+ }
+ if (operatorChain != null) {
+ operatorChain.releaseOutputs();
+ }
+ }
+ }
}
@Override
@@ -188,6 +204,9 @@ public abstract class StreamTask<OUT, Operator extends StreamOperator<OUT>>
boolean disposed = false;
try {
+ // first order of business is to ive operators back their state
+ restoreStateLazy();
+
// we need to make sure that any triggers scheduled in open() cannot be
// executed before all operators are opened
synchronized (lock) {
@@ -223,7 +242,7 @@ public abstract class StreamTask<OUT, Operator extends StreamOperator<OUT>>
finally {
isRunning = false;
- timerService.shutdown();
+ timerService.shutdownNow();
// release the output resources. this method should never fail.
if (operatorChain != null) {
@@ -263,7 +282,9 @@ public abstract class StreamTask<OUT, Operator extends StreamOperator<OUT>>
private void openAllOperators() throws Exception {
for (StreamOperator<?> operator : operatorChain.getAllOperators()) {
- operator.open();
+ if (operator != null) {
+ operator.open();
+ }
}
}
@@ -272,20 +293,27 @@ public abstract class StreamTask<OUT, Operator extends StreamOperator<OUT>>
// elements in their close methods.
StreamOperator<?>[] allOperators = operatorChain.getAllOperators();
for (int i = allOperators.length - 1; i >= 0; i--) {
- allOperators[i].close();
+ StreamOperator<?> operator = allOperators[i];
+ if (operator != null) {
+ operator.close();
+ }
}
}
private void tryDisposeAllOperators() throws Exception {
for (StreamOperator<?> operator : operatorChain.getAllOperators()) {
- operator.dispose();
+ if (operator != null) {
+ operator.dispose();
+ }
}
}
private void disposeAllOperators() {
for (StreamOperator<?> operator : operatorChain.getAllOperators()) {
try {
- operator.dispose();
+ if (operator != null) {
+ operator.dispose();
+ }
}
catch (Throwable t) {
LOG.error("Error during disposal of stream operator.", t);
@@ -354,22 +382,36 @@ public abstract class StreamTask<OUT, Operator extends StreamOperator<OUT>>
// ------------------------------------------------------------------------
@Override
- public void setInitialState(StreamTaskStateList initialState) throws Exception {
- LOG.info("Restoring checkpointed state to task {}", getName());
-
- final StreamOperator<?>[] allOperators = operatorChain.getAllOperators();
- final StreamTaskState[] states = initialState.getState(userClassLoader);
-
- for (int i = 0; i < states.length; i++) {
- StreamTaskState state = states[i];
- StreamOperator<?> operator = allOperators[i];
+ public void setInitialState(StreamTaskStateList initialState) {
+ lazyRestoreState = initialState;
+ }
+
+ public void restoreStateLazy() throws Exception {
+ if (lazyRestoreState != null) {
+ LOG.info("Restoring checkpointed state to task {}", getName());
- if (state != null && operator != null) {
- LOG.debug("Task {} in chain ({}) has checkpointed state", i, getName());
- operator.restoreState(state);
+ try {
+ final StreamOperator<?>[] allOperators = operatorChain.getAllOperators();
+ final StreamTaskState[] states = lazyRestoreState.getState(userClassLoader);
+
+ // be GC friendly
+ lazyRestoreState = null;
+
+ for (int i = 0; i < states.length; i++) {
+ StreamTaskState state = states[i];
+ StreamOperator<?> operator = allOperators[i];
+
+ if (state != null && operator != null) {
+ LOG.debug("Task {} in chain ({}) has checkpointed state", i, getName());
+ operator.restoreState(state);
+ }
+ else if (operator != null) {
+ LOG.debug("Task {} in chain ({}) does not have checkpointed state", i, getName());
+ }
+ }
}
- else if (operator != null) {
- LOG.debug("Task {} in chain ({}) does not have checkpointed state", i, getName());
+ catch (Exception e) {
+ throw new Exception("Could not restore checkpointed state to operators and functions", e);
}
}
}
@@ -380,24 +422,27 @@ public abstract class StreamTask<OUT, Operator extends StreamOperator<OUT>>
synchronized (lock) {
if (isRunning) {
-
- // since both state checkpointing and downstream barrier emission occurs in this
- // lock scope, they are an atomic operation regardless of the order in which they occur
- // we immediately emit the checkpoint barriers, so the downstream operators can start
- // their checkpoint work as soon as possible
- operatorChain.broadcastCheckpointBarrier(checkpointId, timestamp);
-
// now draw the state snapshot
try {
final StreamOperator<?>[] allOperators = operatorChain.getAllOperators();
final StreamTaskState[] states = new StreamTaskState[allOperators.length];
for (int i = 0; i < states.length; i++) {
- StreamTaskState state = allOperators[i].snapshotOperatorState(checkpointId, timestamp);
- states[i] = state.isEmpty() ? null : state;
+ StreamOperator<?> operator = allOperators[i];
+ if (operator != null) {
+ StreamTaskState state = operator.snapshotOperatorState(checkpointId, timestamp);
+ states[i] = state.isEmpty() ? null : state;
+ }
}
StreamTaskStateList allStates = new StreamTaskStateList(states);
+
+ // since both state checkpointing and downstream barrier emission occurs in this
+ // lock scope, they are an atomic operation regardless of the order in which they occur
+ // we immediately emit the checkpoint barriers, so the downstream operators can start
+ // their checkpoint work as soon as possible
+ operatorChain.broadcastCheckpointBarrier(checkpointId, timestamp);
+
if (allStates.isEmpty()) {
getEnvironment().acknowledgeCheckpoint(checkpointId);
} else {
@@ -420,7 +465,9 @@ public abstract class StreamTask<OUT, Operator extends StreamOperator<OUT>>
LOG.debug("Notification of complete checkpoint for task {}", getName());
for (StreamOperator<?> operator : operatorChain.getAllOperators()) {
- operator.notifyOfCompletedCheckpoint(checkpointId);
+ if (operator != null) {
+ operator.notifyOfCompletedCheckpoint(checkpointId);
+ }
}
}
else {
[08/24] flink git commit: [FLINK-2808] [streaming] Refactor and
extend state backend abstraction
Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java
index dc9a152..7a1bea4 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java
@@ -19,26 +19,17 @@
package org.apache.flink.streaming.api.operators;
import java.io.Serializable;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Objects;
import org.apache.flink.api.common.functions.Function;
import org.apache.flink.api.common.functions.util.FunctionUtils;
-import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.runtime.state.StateHandle;
-import org.apache.flink.runtime.state.StateHandleProvider;
import org.apache.flink.streaming.api.checkpoint.CheckpointNotifier;
import org.apache.flink.streaming.api.checkpoint.Checkpointed;
-import org.apache.flink.streaming.api.state.OperatorStateHandle;
-import org.apache.flink.streaming.api.state.PartitionedStreamOperatorState;
-import org.apache.flink.streaming.api.state.StreamOperatorState;
-import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import org.apache.flink.streaming.api.state.StateBackend;
+import org.apache.flink.streaming.runtime.tasks.StreamTaskState;
+
+import static java.util.Objects.requireNonNull;
/**
* This is used as the base class for operators that have a user-defined
@@ -50,22 +41,20 @@ import org.slf4j.LoggerFactory;
* @param <F>
* The type of the user function
*/
-public abstract class AbstractUdfStreamOperator<OUT, F extends Function>
- extends AbstractStreamOperator<OUT> implements StatefulStreamOperator<OUT> {
+public abstract class AbstractUdfStreamOperator<OUT, F extends Function> extends AbstractStreamOperator<OUT> {
private static final long serialVersionUID = 1L;
- private static final Logger LOG = LoggerFactory.getLogger(AbstractUdfStreamOperator.class);
-
+
/** the user function */
protected final F userFunction;
/** Flag to prevent duplicate function.close() calls in close() and dispose() */
- private boolean functionsClosed = false;
-
+ private transient boolean functionsClosed = false;
+
public AbstractUdfStreamOperator(F userFunction) {
- this.userFunction = Objects.requireNonNull(userFunction);
+ this.userFunction = requireNonNull(userFunction);
}
/**
@@ -79,18 +68,13 @@ public abstract class AbstractUdfStreamOperator<OUT, F extends Function>
// ------------------------------------------------------------------------
// operator life cycle
// ------------------------------------------------------------------------
-
- @Override
- public final void setup(Output<StreamRecord<OUT>> output, StreamingRuntimeContext runtimeContext) {
- super.setup(output, runtimeContext);
- FunctionUtils.setFunctionRuntimeContext(userFunction, runtimeContext);
- }
-
@Override
- public void open(Configuration parameters) throws Exception {
- super.open(parameters);
- FunctionUtils.openFunction(userFunction, parameters);
+ public void open() throws Exception {
+ super.open();
+
+ FunctionUtils.setFunctionRuntimeContext(userFunction, getRuntimeContext());
+ FunctionUtils.openFunction(userFunction, new Configuration());
}
@Override
@@ -118,76 +102,81 @@ public abstract class AbstractUdfStreamOperator<OUT, F extends Function>
// ------------------------------------------------------------------------
@Override
- @SuppressWarnings({ "unchecked", "rawtypes" })
- public void restoreInitialState(Tuple2<StateHandle<Serializable>, Map<String, OperatorStateHandle>> snapshots) throws Exception {
+ public StreamTaskState snapshotOperatorState(long checkpointId, long timestamp) throws Exception {
+ StreamTaskState state = super.snapshotOperatorState(checkpointId, timestamp);
- // Restore state using the Checkpointed interface
- if (userFunction instanceof Checkpointed && snapshots.f0 != null) {
- ((Checkpointed) userFunction).restoreState(snapshots.f0.getState(runtimeContext.getUserCodeClassLoader()));
- }
-
- if (snapshots.f1 != null) {
- // We iterate over the states registered for this operator, initialize and restore it
- for (Entry<String, OperatorStateHandle> snapshot : snapshots.f1.entrySet()) {
- StreamOperatorState restoredOpState = runtimeContext.getState(snapshot.getKey(), snapshot.getValue().isPartitioned());
- StateHandle<Serializable> checkpointHandle = snapshot.getValue();
- restoredOpState.restoreState(checkpointHandle, runtimeContext.getUserCodeClassLoader());
+ if (userFunction instanceof Checkpointed) {
+ @SuppressWarnings("unchecked")
+ Checkpointed<Serializable> chkFunction = (Checkpointed<Serializable>) userFunction;
+
+ Serializable udfState;
+ try {
+ udfState = chkFunction.snapshotState(checkpointId, timestamp);
+ }
+ catch (Exception e) {
+ throw new Exception("Failed to draw state snapshot from function: " + e.getMessage(), e);
+ }
+
+ if (udfState != null) {
+ try {
+ StateBackend<?> stateBackend = getStateBackend();
+ StateHandle<Serializable> handle =
+ stateBackend.checkpointStateSerializable(udfState, checkpointId, timestamp);
+ state.setFunctionState(handle);
+ }
+ catch (Exception e) {
+ throw new Exception("Failed to add the state snapshot of the function to the checkpoint: "
+ + e.getMessage(), e);
+ }
}
}
+ return state;
}
- @SuppressWarnings({ "rawtypes", "unchecked" })
- public Tuple2<StateHandle<Serializable>, Map<String, OperatorStateHandle>> getStateSnapshotFromFunction(long checkpointId, long timestamp)
- throws Exception {
- // Get all the states for the operator
- Map<String, StreamOperatorState<?, ?>> operatorStates = runtimeContext.getOperatorStates();
+ @Override
+ public void restoreState(StreamTaskState state) throws Exception {
+ super.restoreState(state);
- Map<String, OperatorStateHandle> operatorStateSnapshots;
- if (operatorStates.isEmpty()) {
- // We return null to signal that there is nothing to checkpoint
- operatorStateSnapshots = null;
- } else {
- // Checkpoint the states and store the handles in a map
- Map<String, OperatorStateHandle> snapshots = new HashMap<String, OperatorStateHandle>();
-
- for (Entry<String, StreamOperatorState<?, ?>> state : operatorStates.entrySet()) {
- boolean isPartitioned = state.getValue() instanceof PartitionedStreamOperatorState;
- snapshots.put(state.getKey(),
- new OperatorStateHandle(state.getValue().snapshotState(checkpointId, timestamp),
- isPartitioned));
- }
-
- operatorStateSnapshots = snapshots;
- }
+ StateHandle<Serializable> stateHandle = state.getFunctionState();
- StateHandle<Serializable> checkpointedSnapshot = null;
- // if the UDF implements the Checkpointed interface we draw a snapshot
- if (userFunction instanceof Checkpointed) {
- StateHandleProvider<Serializable> provider = runtimeContext.getStateHandleProvider();
- Serializable state = ((Checkpointed) userFunction).snapshotState(checkpointId, timestamp);
- if (state != null) {
- checkpointedSnapshot = provider.createStateHandle(state);
+ if (userFunction instanceof Checkpointed && stateHandle != null) {
+ @SuppressWarnings("unchecked")
+ Checkpointed<Serializable> chkFunction = (Checkpointed<Serializable>) userFunction;
+
+ Serializable functionState = stateHandle.getState(getUserCodeClassloader());
+ if (functionState != null) {
+ try {
+ chkFunction.restoreState(functionState);
+ }
+ catch (Exception e) {
+ throw new Exception("Failed to restore state to function: " + e.getMessage(), e);
+ }
}
}
-
- // if we have either operator or checkpointed state we store it in a
- // tuple2 otherwise return null
- if (operatorStateSnapshots != null || checkpointedSnapshot != null) {
- return Tuple2.of(checkpointedSnapshot, operatorStateSnapshots);
- } else {
- return null;
- }
-
}
- public void notifyCheckpointComplete(long checkpointId) throws Exception {
+ @Override
+ public void notifyOfCompletedCheckpoint(long checkpointId) throws Exception {
+ super.notifyOfCompletedCheckpoint(checkpointId);
+
if (userFunction instanceof CheckpointNotifier) {
- try {
- ((CheckpointNotifier) userFunction).notifyCheckpointComplete(checkpointId);
- } catch (Exception e) {
- throw new Exception("Error while confirming checkpoint " + checkpointId + " to the stream function", e);
- }
+ ((CheckpointNotifier) userFunction).notifyCheckpointComplete(checkpointId);
}
}
+
+ // ------------------------------------------------------------------------
+ // Utilities
+ // ------------------------------------------------------------------------
+
+ /**
+ *
+ * Since the streaming API does not implement any parametrization of functions via a
+ * configuration, the config returned here is actually empty.
+ *
+ * @return The user function parameters (currently empty)
+ */
+ public Configuration getUserFunctionParameters() {
+ return new Configuration();
+ }
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/ChainingStrategy.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/ChainingStrategy.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/ChainingStrategy.java
new file mode 100644
index 0000000..3a752b0
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/ChainingStrategy.java
@@ -0,0 +1,47 @@
+/*
+ * 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.operators;
+
+/**
+ * Defines the chaining scheme for the operator.
+ * By default {@link #ALWAYS} is used, which means operators will be eagerly chained whenever possible.
+ */
+public enum ChainingStrategy {
+
+ /**
+ * Chaining will happen even if chaining is disabled on the execution environment.
+ * This should only be used by system-level operators, not operators implemented by users.
+ */
+ FORCE_ALWAYS,
+
+ /**
+ * Operators will be eagerly chained whenever possible, for
+ * maximal performance. It is generally a good practice to allow maximal
+ * chaining and increase operator parallelism
+ */
+ ALWAYS,
+
+ /**
+ * The operator will not be chained to the preceding or succeeding operators.
+ */
+ NEVER,
+
+
+ HEAD
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/OneInputStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/OneInputStreamOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/OneInputStreamOperator.java
index 7ca540f..705c1b3 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/OneInputStreamOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/OneInputStreamOperator.java
@@ -35,7 +35,7 @@ public interface OneInputStreamOperator<IN, OUT> extends StreamOperator<OUT> {
* Processes one element that arrived at this operator.
* This method is guaranteed to not be called concurrently with other methods of the operator.
*/
- public void processElement(StreamRecord<IN> element) throws Exception;
+ void processElement(StreamRecord<IN> element) throws Exception;
/**
* Processes a {@link Watermark}.
@@ -43,5 +43,5 @@ public interface OneInputStreamOperator<IN, OUT> extends StreamOperator<OUT> {
*
* @see org.apache.flink.streaming.api.watermark.Watermark
*/
- public void processWatermark(Watermark mark) throws Exception;
+ void processWatermark(Watermark mark) throws Exception;
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/Output.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/Output.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/Output.java
index b68432604..0cbc954 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/Output.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/Output.java
@@ -34,7 +34,7 @@ public interface Output<T> extends Collector<T> {
* operators.
*
* <p>A watermark specifies that no element with a timestamp older or equal to the watermark
- * timestamp will be emitted in the future.</p>
+ * timestamp will be emitted in the future.
*/
void emitWatermark(Watermark mark);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StatefulStreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StatefulStreamOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StatefulStreamOperator.java
deleted file mode 100644
index d400fc4..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StatefulStreamOperator.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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.operators;
-
-import java.io.Serializable;
-import java.util.Map;
-
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.runtime.state.StateHandle;
-import org.apache.flink.streaming.api.state.OperatorStateHandle;
-
-/**
- * Interface for Stream operators that can have state. This interface is used for checkpointing
- * and restoring that state.
- *
- * @param <OUT> The output type of the operator
- */
-public interface StatefulStreamOperator<OUT> extends StreamOperator<OUT> {
-
- void restoreInitialState(Tuple2<StateHandle<Serializable>, Map<String, OperatorStateHandle>> state) throws Exception;
-
- Tuple2<StateHandle<Serializable>, Map<String, OperatorStateHandle>> getStateSnapshotFromFunction(long checkpointId, long timestamp) throws Exception;
-
- void notifyCheckpointComplete(long checkpointId) throws Exception;
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamFlatMap.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamFlatMap.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamFlatMap.java
index ff7f662..23b638e 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamFlatMap.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamFlatMap.java
@@ -18,7 +18,6 @@
package org.apache.flink.streaming.api.operators;
import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
@@ -36,8 +35,8 @@ public class StreamFlatMap<IN, OUT>
}
@Override
- public void open(Configuration parameters) throws Exception {
- super.open(parameters);
+ public void open() throws Exception {
+ super.open();
collector = new TimestampedCollector<OUT>(output);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedFold.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedFold.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedFold.java
index 732630a..79e319a 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedFold.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedFold.java
@@ -22,53 +22,41 @@ import java.io.ByteArrayOutputStream;
import java.io.DataInputStream;
import java.io.DataOutputStream;
import java.io.IOException;
-import java.util.HashMap;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.functions.FoldFunction;
import org.apache.flink.api.common.state.OperatorState;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.memory.InputViewDataInputStreamWrapper;
import org.apache.flink.core.memory.OutputViewDataOutputStreamWrapper;
-import org.apache.flink.streaming.api.state.KVMapCheckpointer;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-public class StreamGroupedFold<IN, OUT> extends AbstractUdfStreamOperator<OUT, FoldFunction<IN, OUT>>
+public class StreamGroupedFold<IN, OUT, KEY>
+ extends AbstractUdfStreamOperator<OUT, FoldFunction<IN, OUT>>
implements OneInputStreamOperator<IN, OUT>, OutputTypeConfigurable<OUT> {
private static final long serialVersionUID = 1L;
// Grouped values
- private KeySelector<IN, ?> keySelector;
- private transient OperatorState<HashMap<Object, OUT>> values;
-
+ private transient OperatorState<OUT> values;
+
+ private transient OUT initialValue;
+
// Initial value serialization
private byte[] serializedInitialValue;
+
private TypeSerializer<OUT> outTypeSerializer;
- private transient OUT initialValue;
-
- // Store the typeinfo, create serializer during runtime
- private TypeInformation<Object> keyTypeInformation;
-
- @SuppressWarnings("unchecked")
- public StreamGroupedFold(FoldFunction<IN, OUT> folder, KeySelector<IN, ?> keySelector,
- OUT initialValue, TypeInformation<IN> inTypeInformation) {
+
+ public StreamGroupedFold(FoldFunction<IN, OUT> folder, OUT initialValue) {
super(folder);
- this.keySelector = keySelector;
this.initialValue = initialValue;
- keyTypeInformation = (TypeInformation<Object>) TypeExtractor
- .getKeySelectorTypes(keySelector, inTypeInformation);
-
}
@Override
- public void open(Configuration configuration) throws Exception {
- super.open(configuration);
+ public void open() throws Exception {
+ super.open();
if (serializedInitialValue == null) {
throw new RuntimeException("No initial value was serialized for the fold " +
@@ -80,25 +68,20 @@ public class StreamGroupedFold<IN, OUT> extends AbstractUdfStreamOperator<OUT, F
new DataInputStream(bais)
);
initialValue = outTypeSerializer.deserialize(in);
-
- values = runtimeContext.getOperatorState("flink_internal_fold_values",
- new HashMap<Object, OUT>(), false,
- new KVMapCheckpointer<>(keyTypeInformation.createSerializer(executionConfig),
- outTypeSerializer));
+ values = createKeyValueState(outTypeSerializer, null);
}
@Override
public void processElement(StreamRecord<IN> element) throws Exception {
- Object key = keySelector.getKey(element.getValue());
- OUT value = values.value().get(key);
+ OUT value = values.value();
if (value != null) {
OUT folded = userFunction.fold(outTypeSerializer.copy(value), element.getValue());
- values.value().put(key, folded);
+ values.update(folded);
output.collect(element.replace(folded));
} else {
OUT first = userFunction.fold(outTypeSerializer.copy(initialValue), element.getValue());
- values.value().put(key, first);
+ values.update(first);
output.collect(element.replace(first));
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedReduce.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedReduce.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedReduce.java
index 579814d..ebc4b09 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedReduce.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamGroupedReduce.java
@@ -19,61 +19,43 @@ package org.apache.flink.streaming.api.operators;
import org.apache.flink.api.common.functions.ReduceFunction;
import org.apache.flink.api.common.state.OperatorState;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.typeutils.TypeExtractor;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.streaming.api.state.KVMapCheckpointer;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import java.util.HashMap;
-
public class StreamGroupedReduce<IN> extends AbstractUdfStreamOperator<IN, ReduceFunction<IN>>
implements OneInputStreamOperator<IN, IN> {
private static final long serialVersionUID = 1L;
+
+ private transient OperatorState<IN> values;
+
+ private TypeSerializer<IN> serializer;
- private KeySelector<IN, ?> keySelector;
- private transient OperatorState<HashMap<Object, IN>> values;
-
- // Store the typeinfo, create serializer during runtime
- private TypeInformation<Object> keyTypeInformation;
- private TypeInformation<IN> valueTypeInformation;
-
- @SuppressWarnings("unchecked")
- public StreamGroupedReduce(ReduceFunction<IN> reducer, KeySelector<IN, ?> keySelector,
- TypeInformation<IN> typeInformation) {
+
+ public StreamGroupedReduce(ReduceFunction<IN> reducer, TypeSerializer<IN> serializer) {
super(reducer);
- this.keySelector = keySelector;
- valueTypeInformation = typeInformation;
- keyTypeInformation = (TypeInformation<Object>) TypeExtractor
- .getKeySelectorTypes(keySelector, typeInformation);
+ this.serializer = serializer;
}
@Override
- public void open(Configuration parameters) throws Exception {
- super.open(parameters);
-
- values = runtimeContext.getOperatorState("flink_internal_reduce_values",
- new HashMap<Object, IN>(), false,
- new KVMapCheckpointer<>(keyTypeInformation.createSerializer(executionConfig),
- valueTypeInformation.createSerializer(executionConfig)));
+ public void open() throws Exception {
+ super.open();
+ values = createKeyValueState(serializer, null);
}
@Override
public void processElement(StreamRecord<IN> element) throws Exception {
- Object key = keySelector.getKey(element.getValue());
-
- IN currentValue = values.value().get(key);
+ IN value = element.getValue();
+ IN currentValue = values.value();
+
if (currentValue != null) {
- // TODO: find a way to let operators copy elements (maybe)
- IN reduced = userFunction.reduce(currentValue, element.getValue());
- values.value().put(key, reduced);
+ IN reduced = userFunction.reduce(currentValue, value);
+ values.update(reduced);
output.collect(element.replace(reduced));
} else {
- values.value().put(key, element.getValue());
- output.collect(element.replace(element.getValue()));
+ values.update(value);
+ output.collect(element.replace(value));
}
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java
index d65dc64..fac26f1 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamOperator.java
@@ -19,9 +19,10 @@ package org.apache.flink.streaming.api.operators;
import java.io.Serializable;
-import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.graph.StreamConfig;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
-import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext;
+import org.apache.flink.streaming.runtime.tasks.StreamTask;
+import org.apache.flink.streaming.runtime.tasks.StreamTaskState;
/**
* Basic interface for stream operators. Implementers would implement one of
@@ -29,27 +30,25 @@ import org.apache.flink.streaming.runtime.tasks.StreamingRuntimeContext;
* {@link org.apache.flink.streaming.api.operators.TwoInputStreamOperator} to create operators
* that process elements.
*
- * <p>
- * The class {@link org.apache.flink.streaming.api.operators.AbstractStreamOperator}
+ * <p> The class {@link org.apache.flink.streaming.api.operators.AbstractStreamOperator}
* offers default implementation for the lifecycle and properties methods.
*
- * <p>
- * Methods of {@code StreamOperator} are guaranteed not to be called concurrently. Also, if using
+ * <p> Methods of {@code StreamOperator} are guaranteed not to be called concurrently. Also, if using
* the timer service, timer callbacks are also guaranteed not to be called concurrently with
* methods on {@code StreamOperator}.
*
* @param <OUT> The output type of the operator
*/
public interface StreamOperator<OUT> extends Serializable {
-
+
// ------------------------------------------------------------------------
- // Life Cycle
+ // life cycle
// ------------------------------------------------------------------------
/**
* Initializes the operator. Sets access to the context and the output.
*/
- void setup(Output<StreamRecord<OUT>> output, StreamingRuntimeContext runtimeContext);
+ void setup(StreamTask<?, ?> containingTask, StreamConfig config, Output<StreamRecord<OUT>> output);
/**
* This method is called immediately before any elements are processed, it should contain the
@@ -57,7 +56,7 @@ public interface StreamOperator<OUT> extends Serializable {
*
* @throws java.lang.Exception An exception in this method causes the operator to fail.
*/
- void open(Configuration config) throws Exception;
+ void open() throws Exception;
/**
* This method is called after all records have been added to the operators via the methods
@@ -82,43 +81,66 @@ public interface StreamOperator<OUT> extends Serializable {
* that the operator has acquired.
*/
void dispose();
-
// ------------------------------------------------------------------------
- // Context and chaining properties
+ // state snapshots
// ------------------------------------------------------------------------
+
+ /**
+ * Called to draw a state snapshot from the operator. This method snapshots the operator state
+ * (if the operator is stateful) and the key/value state (if it is being used and has been
+ * initialized).
+ *
+ * @param checkpointId The ID of the checkpoint.
+ * @param timestamp The timestamp of the checkpoint.
+ *
+ * @return The StreamTaskState object, possibly containing the snapshots for the
+ * operator and key/value state.
+ *
+ * @throws Exception Forwards exceptions that occur while drawing snapshots from the operator
+ * and the key/value state.
+ */
+ StreamTaskState snapshotOperatorState(long checkpointId, long timestamp) throws Exception;
/**
- * Returns a context that allows the operator to query information about the execution and also
- * to interact with systems such as broadcast variables and managed state. This also allows
- * to register timers.
+ * Restores the operator state, if this operator's execution is recovering from a checkpoint.
+ * This method restores the operator state (if the operator is stateful) and the key/value state
+ * (if it had been used and was initialized when the snapshot ocurred).
+ *
+ * <p>This method is called after {@link #setup(StreamTask, StreamConfig, Output)}
+ * and before {@link #open()}.
+ *
+ * @param state The state of operator that was snapshotted as part of checkpoint
+ * from which the execution is restored.
+ *
+ * @throws Exception Exceptions during state restore should be forwarded, so that the system can
+ * properly react to failed state restore and fail the execution attempt.
*/
- StreamingRuntimeContext getRuntimeContext();
+ void restoreState(StreamTaskState state) throws Exception;
/**
+ * Called when the checkpoint with the given ID is completed and acknowledged on the JobManager.
+ *
+ * @param checkpointId The ID of the checkpoint that has been completed.
+ *
+ * @throws Exception Exceptions during checkpoint acknowledgement may be forwarded and will cause
+ * the program to fail and enter recovery.
+ */
+ void notifyOfCompletedCheckpoint(long checkpointId) throws Exception;
+
+ // ------------------------------------------------------------------------
+ // miscellaneous
+ // ------------------------------------------------------------------------
+
+ void setKeyContextElement(StreamRecord<?> record) throws Exception;
+
+ /**
* An operator can return true here to disable copying of its input elements. This overrides
* the object-reuse setting on the {@link org.apache.flink.api.common.ExecutionConfig}
*/
boolean isInputCopyingDisabled();
-
- void setChainingStrategy(ChainingStrategy strategy);
-
+
ChainingStrategy getChainingStrategy();
- /**
- * Defines the chaining scheme for the operator. By default <b>ALWAYS</b> is used,
- * which means operators will be eagerly chained whenever possible, for
- * maximal performance. It is generally a good practice to allow maximal
- * chaining and increase operator parallelism. </p> When the strategy is set
- * to <b>NEVER</b>, the operator will not be chained to the preceding or succeeding
- * operators.</p> <b>HEAD</b> strategy marks a start of a new chain, so that the
- * operator will not be chained to preceding operators, only succeding ones.
- *
- * <b>FORCE_ALWAYS</b> will enable chaining even if chaining is disabled on the execution
- * environment. This should only be used by system-level operators, not operators implemented
- * by users.
- */
- public static enum ChainingStrategy {
- FORCE_ALWAYS, ALWAYS, NEVER, HEAD
- }
+ void setChainingStrategy(ChainingStrategy strategy);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamProject.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamProject.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamProject.java
index c0815b5..1ce4ff6 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamProject.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamProject.java
@@ -19,7 +19,6 @@ package org.apache.flink.streaming.api.operators;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.java.tuple.Tuple;
-import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
@@ -53,8 +52,8 @@ public class StreamProject<IN, OUT extends Tuple>
}
@Override
- public void open(Configuration config) throws Exception {
- super.open(config);
+ public void open() throws Exception {
+ super.open();
outTuple = outSerializer.createInstance();
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java
index ecf799b..fbecbd1 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamSource.java
@@ -43,7 +43,8 @@ public class StreamSource<T> extends AbstractUdfStreamOperator<T, SourceFunction
}
public void run(final Object lockingObject, final Output<StreamRecord<T>> collector) throws Exception {
-
+ final ExecutionConfig executionConfig = getExecutionConfig();
+
if (userFunction instanceof EventTimeSourceFunction) {
ctx = new ManualWatermarkContext<T>(lockingObject, collector);
} else if (executionConfig.getAutoWatermarkInterval() > 0) {
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java
new file mode 100644
index 0000000..40998dd
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java
@@ -0,0 +1,162 @@
+/*
+ * 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.operators;
+
+import org.apache.flink.api.common.accumulators.Accumulator;
+import org.apache.flink.api.common.functions.BroadcastVariableInitializer;
+import org.apache.flink.api.common.functions.util.AbstractRuntimeUDFContext;
+import org.apache.flink.api.common.state.OperatorState;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.TypeExtractor;
+import org.apache.flink.runtime.execution.Environment;
+import org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider;
+import org.apache.flink.streaming.runtime.operators.Triggerable;
+
+import java.util.List;
+import java.util.Map;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * Implementation of the {@link org.apache.flink.api.common.functions.RuntimeContext},
+ * for streaming operators.
+ */
+public class StreamingRuntimeContext extends AbstractRuntimeUDFContext {
+
+ /** The operator to which this function belongs */
+ private final AbstractStreamOperator<?> operator;
+
+ /** The task environment running the operator */
+ private final Environment taskEnvironment;
+
+ /** The key/value state, if the user-function requests it */
+ private OperatorState<?> keyValueState;
+
+ /** Type of the values stored in the state, to make sure repeated requests of the state are consistent */
+ private TypeInformation<?> stateTypeInfo;
+
+
+ public StreamingRuntimeContext(AbstractStreamOperator<?> operator,
+ Environment env, Map<String, Accumulator<?, ?>> accumulators) {
+ super(env.getTaskName(),
+ env.getNumberOfSubtasks(),
+ env.getIndexInSubtaskGroup(),
+ env.getUserClassLoader(),
+ operator.getExecutionConfig(),
+ accumulators,
+ env.getDistributedCacheEntries());
+
+ this.operator = operator;
+ this.taskEnvironment = env;
+ }
+
+ // ------------------------------------------------------------------------
+
+ /**
+ * Returns the input split provider associated with the operator.
+ *
+ * @return The input split provider.
+ */
+ public InputSplitProvider getInputSplitProvider() {
+ return taskEnvironment.getInputSplitProvider();
+ }
+
+ /**
+ * Register a timer callback. At the specified time the {@link Triggerable } will be invoked.
+ * This call is guaranteed to not happen concurrently with method calls on the operator.
+ *
+ * @param time The absolute time in milliseconds.
+ * @param target The target to be triggered.
+ */
+ public void registerTimer(long time, Triggerable target) {
+ operator.registerTimer(time, target);
+ }
+
+ // ------------------------------------------------------------------------
+ // broadcast variables
+ // ------------------------------------------------------------------------
+
+ @Override
+ public <RT> List<RT> getBroadcastVariable(String name) {
+ throw new UnsupportedOperationException("Broadcast variables can only be used in DataSet programs");
+ }
+
+ @Override
+ public <T, C> C getBroadcastVariableWithInitializer(String name, BroadcastVariableInitializer<T, C> initializer) {
+ throw new UnsupportedOperationException("Broadcast variables can only be used in DataSet programs");
+ }
+
+ // ------------------------------------------------------------------------
+ // key/value state
+ // ------------------------------------------------------------------------
+
+ @Override
+ public <S> OperatorState<S> getKeyValueState(Class<S> stateType, S defaultState) {
+ requireNonNull(stateType, "The state type class must not be null");
+
+ TypeInformation<S> typeInfo;
+ try {
+ typeInfo = TypeExtractor.getForClass(stateType);
+ }
+ catch (Exception e) {
+ throw new RuntimeException("Cannot analyze type '" + stateType.getName() +
+ "' from the class alone, due to generic type parameters. " +
+ "Please specify the TypeInformation directly.");
+ }
+
+ return getKeyValueState(typeInfo, defaultState);
+ }
+
+ @Override
+ public <S> OperatorState<S> getKeyValueState(TypeInformation<S> stateType, S defaultState) {
+ requireNonNull(stateType, "The state type information must not be null");
+
+ // check if this is a repeated call to access the state
+ if (this.stateTypeInfo != null && this.keyValueState != null) {
+ // repeated call
+ if (this.stateTypeInfo.equals(stateType)) {
+ // valid case, same type requested again
+ @SuppressWarnings("unchecked")
+ OperatorState<S> previous = (OperatorState<S>) this.keyValueState;
+ return previous;
+ }
+ else {
+ // invalid case, different type requested this time
+ throw new IllegalStateException("Cannot initialize key/value state for type " + stateType +
+ " ; The key/value state has already been created and initialized for a different type: " +
+ this.stateTypeInfo);
+ }
+ }
+ else {
+ // first time access to the key/value state
+ try {
+ OperatorState<S> state = operator.createKeyValueState(stateType, defaultState);
+ this.keyValueState = state;
+ this.stateTypeInfo = stateType;
+ return state;
+ }
+ catch (RuntimeException e) {
+ throw e;
+ }
+ catch (Exception e) {
+ throw new RuntimeException("Cannot initialize the key/value state", e);
+ }
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMap.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMap.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMap.java
index cbf59c1..806cef2 100644
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMap.java
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMap.java
@@ -17,7 +17,6 @@
package org.apache.flink.streaming.api.operators.co;
-import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction;
import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
@@ -44,8 +43,8 @@ public class CoStreamFlatMap<IN1, IN2, OUT>
}
@Override
- public void open(Configuration parameters) throws Exception {
- super.open(parameters);
+ public void open() throws Exception {
+ super.open();
collector = new TimestampedCollector<OUT>(output);
}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/AbstractHeapKvState.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/AbstractHeapKvState.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/AbstractHeapKvState.java
new file mode 100644
index 0000000..b974674
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/AbstractHeapKvState.java
@@ -0,0 +1,145 @@
+/*
+ * 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.state;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.memory.DataOutputView;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import static java.util.Objects.requireNonNull;
+
+/**
+ * Base class for key/value state implementations that are backed by a regular heap hash map. The
+ * concrete implementations define how the state is checkpointed.
+ *
+ * @param <K> The type of the key.
+ * @param <V> The type of the value.
+ * @param <Backend> The type of the backend that snapshots this key/value state.
+ */
+public abstract class AbstractHeapKvState<K, V, Backend extends StateBackend<Backend>> implements KvState<K, V, Backend> {
+
+ /** Map containing the actual key/value pairs */
+ private final HashMap<K, V> state;
+
+ /** The serializer for the keys */
+ private final TypeSerializer<K> keySerializer;
+
+ /** The serializer for the values */
+ private final TypeSerializer<V> valueSerializer;
+
+ /** The value that is returned when no other value has been associated with a key, yet */
+ private final V defaultValue;
+
+ /** The current key, which the next value methods will refer to */
+ private K currentKey;
+
+ /**
+ * Creates a new empty key/value state.
+ *
+ * @param keySerializer The serializer for the keys.
+ * @param valueSerializer The serializer for the values.
+ * @param defaultValue The value that is returned when no other value has been associated with a key, yet.
+ */
+ protected AbstractHeapKvState(TypeSerializer<K> keySerializer,
+ TypeSerializer<V> valueSerializer,
+ V defaultValue) {
+ this(keySerializer, valueSerializer, defaultValue, new HashMap<K, V>());
+ }
+
+ /**
+ * Creates a new key/value state for the given hash map of key/value pairs.
+ *
+ * @param keySerializer The serializer for the keys.
+ * @param valueSerializer The serializer for the values.
+ * @param defaultValue The value that is returned when no other value has been associated with a key, yet.
+ * @param state The state map to use in this kev/value state. May contain initial state.
+ */
+ protected AbstractHeapKvState(TypeSerializer<K> keySerializer,
+ TypeSerializer<V> valueSerializer,
+ V defaultValue,
+ HashMap<K, V> state) {
+ this.state = requireNonNull(state);
+ this.keySerializer = requireNonNull(keySerializer);
+ this.valueSerializer = requireNonNull(valueSerializer);
+ this.defaultValue = defaultValue;
+ }
+
+ // ------------------------------------------------------------------------
+
+ @Override
+ public V value() {
+ V value = state.get(currentKey);
+ return value != null ? value : defaultValue;
+ }
+
+ @Override
+ public void update(V value) {
+ if (value != null) {
+ state.put(currentKey, value);
+ }
+ else {
+ state.remove(currentKey);
+ }
+ }
+
+ @Override
+ public void setCurrentKey(K currentKey) {
+ this.currentKey = currentKey;
+ }
+
+ @Override
+ public int size() {
+ return state.size();
+ }
+
+ @Override
+ public void dispose() {
+ state.clear();
+ }
+
+ /**
+ * Gets the serializer for the keys.
+ * @return The serializer for the keys.
+ */
+ public TypeSerializer<K> getKeySerializer() {
+ return keySerializer;
+ }
+
+ /**
+ * Gets the serializer for the values.
+ * @return The serializer for the values.
+ */
+ public TypeSerializer<V> getValueSerializer() {
+ return valueSerializer;
+ }
+
+ // ------------------------------------------------------------------------
+ // checkpointing utilities
+ // ------------------------------------------------------------------------
+
+ protected void writeStateToOutputView(final DataOutputView out) throws IOException {
+ for (Map.Entry<K, V> entry : state.entrySet()) {
+ keySerializer.serialize(entry.getKey(), out);
+ valueSerializer.serialize(entry.getValue(), out);
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/BasicCheckpointer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/BasicCheckpointer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/BasicCheckpointer.java
deleted file mode 100644
index 14d1504..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/BasicCheckpointer.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * 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.state;
-
-import java.io.Serializable;
-
-import org.apache.flink.api.common.state.StateCheckpointer;
-
-public class BasicCheckpointer implements StateCheckpointer<Serializable, Serializable> {
-
- @Override
- public Serializable snapshotState(Serializable state, long checkpointId, long checkpointTimestamp) {
- return state;
- }
-
- @Override
- public Serializable restoreState(Serializable stateSnapshot) {
- return stateSnapshot;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/EagerStateStore.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/EagerStateStore.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/EagerStateStore.java
deleted file mode 100644
index 2091624..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/EagerStateStore.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * 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.state;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.flink.api.common.state.StateCheckpointer;
-import org.apache.flink.runtime.state.StateHandle;
-import org.apache.flink.runtime.state.StateHandleProvider;
-
-public class EagerStateStore<S, C extends Serializable> implements PartitionedStateStore<S, C> {
-
- private StateCheckpointer<S, C> checkpointer;
- private final StateHandleProvider<Serializable> provider;
-
- private Map<Serializable, S> fetchedState;
-
- @SuppressWarnings("unchecked")
- public EagerStateStore(StateCheckpointer<S, C> checkpointer, StateHandleProvider<C> provider) {
- this.checkpointer = checkpointer;
- this.provider = (StateHandleProvider<Serializable>) provider;
-
- fetchedState = new HashMap<Serializable, S>();
- }
-
- @Override
- public S getStateForKey(Serializable key) throws IOException {
- return fetchedState.get(key);
- }
-
- @Override
- public void setStateForKey(Serializable key, S state) {
- fetchedState.put(key, state);
- }
-
- @Override
- public void removeStateForKey(Serializable key) {
- fetchedState.remove(key);
- }
-
- @Override
- public Map<Serializable, S> getPartitionedState() throws IOException {
- return fetchedState;
- }
-
- @Override
- public StateHandle<Serializable> snapshotStates(long checkpointId, long checkpointTimestamp) {
- // we map the values in the state-map using the state-checkpointer and store it as a checkpoint
- Map<Serializable, C> checkpoints = new HashMap<Serializable, C>();
- for (Entry<Serializable, S> stateEntry : fetchedState.entrySet()) {
- checkpoints.put(stateEntry.getKey(),
- checkpointer.snapshotState(stateEntry.getValue(), checkpointId, checkpointTimestamp));
- }
- return provider.createStateHandle((Serializable) checkpoints);
- }
-
- @Override
- public void restoreStates(StateHandle<Serializable> snapshot, ClassLoader userCodeClassLoader)
- throws Exception {
-
- @SuppressWarnings("unchecked")
- Map<Serializable, C> checkpoints = (Map<Serializable, C>) snapshot.getState(userCodeClassLoader);
-
- // we map the values back to the state from the checkpoints
- for (Entry<Serializable, C> snapshotEntry : checkpoints.entrySet()) {
- fetchedState.put(snapshotEntry.getKey(), (S) checkpointer.restoreState(snapshotEntry.getValue()));
- }
- }
-
- @Override
- public boolean containsKey(Serializable key) {
- return fetchedState.containsKey(key);
- }
-
- @Override
- public void setCheckPointer(StateCheckpointer<S, C> checkpointer) {
- this.checkpointer = checkpointer;
- }
-
- @Override
- public String toString() {
- return fetchedState.toString();
- }
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/KVMapCheckpointer.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/KVMapCheckpointer.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/KVMapCheckpointer.java
deleted file mode 100644
index 17cb6a0..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/KVMapCheckpointer.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * 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.state;
-
-import org.apache.flink.api.common.state.StateCheckpointer;
-import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.typeutils.runtime.ByteArrayInputView;
-import org.apache.flink.core.memory.DataOutputView;
-import org.apache.flink.core.memory.OutputViewDataOutputStreamWrapper;
-
-import java.io.ByteArrayOutputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * Implementation of the {@link StateCheckpointer} interface for a map storing
- * types compatible with Flink's serialization system.
- *
- * @param <K> key type
- * @param <V> value type
- */
-public class KVMapCheckpointer<K, V> implements StateCheckpointer<HashMap<K, V>, byte[]> {
-
- private TypeSerializer<K> keySerializer;
- private TypeSerializer<V> valueSerializer;
-
- public KVMapCheckpointer(TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer) {
- this.keySerializer = keySerializer;
- this.valueSerializer = valueSerializer;
- }
-
- @Override
- public byte[] snapshotState(HashMap<K, V> stateMap, long checkpointId, long checkpointTimestamp) {
- ByteArrayOutputStream bos = new ByteArrayOutputStream(stateMap.size() * 16);
- DataOutputView out = new OutputViewDataOutputStreamWrapper(new DataOutputStream(bos));
- try {
- out.writeInt(stateMap.size());
- for (Map.Entry<K, V> kv : stateMap.entrySet()) {
- keySerializer.serialize(kv.getKey(), out);
- valueSerializer.serialize(kv.getValue(), out);
- }
- } catch (IOException e) {
- throw new RuntimeException("Failed to write snapshot", e);
- }
- return bos.toByteArray();
- }
-
- @Override
- public HashMap<K, V> restoreState(byte[] stateSnapshot) {
- ByteArrayInputView in = new ByteArrayInputView(stateSnapshot);
-
- HashMap<K, V> returnMap = new HashMap<>();
- try {
- int size = in.readInt();
- for (int i = 0; i < size; i++) {
- returnMap.put(keySerializer.deserialize(in), valueSerializer.deserialize(in));
- }
- } catch (IOException e) {
- throw new RuntimeException("Failed to read snapshot", e);
- }
-
- return returnMap;
- }
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/KvState.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/KvState.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/KvState.java
new file mode 100644
index 0000000..9c628f8
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/KvState.java
@@ -0,0 +1,69 @@
+/*
+ * 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.state;
+
+import org.apache.flink.api.common.state.OperatorState;
+
+/**
+ * Key/Value state implementation for user-defined state. The state is backed by a state
+ * backend, which typically follows one of the following patterns: Either the state is stored
+ * in the key/value state object directly (meaning in the executing JVM) and snapshotted by the
+ * state backend into some store (during checkpoints), or the key/value state is in fact backed
+ * by an external key/value store as the state backend, and checkpoints merely record the
+ * metadata of what is considered part of the checkpoint.
+ *
+ * @param <K> The type of the key.
+ * @param <V> The type of the value.
+ */
+public interface KvState<K, V, Backend extends StateBackend<Backend>> extends OperatorState<V> {
+
+ /**
+ * Sets the current key, which will be used to retrieve values for the next calls to
+ * {@link #value()} and {@link #update(Object)}.
+ *
+ * @param key The key.
+ */
+ void setCurrentKey(K key);
+
+ /**
+ * Creates a snapshot of this state.
+ *
+ * @param checkpointId The ID of the checkpoint for which the snapshot should be created.
+ * @param timestamp The timestamp of the checkpoint.
+ * @return A snapshot handle for this key/value state.
+ *
+ * @throws Exception Exceptions during snapshotting the state should be forwarded, so the system
+ * can react to failed snapshots.
+ */
+ KvStateSnapshot<K, V, Backend> shapshot(long checkpointId, long timestamp) throws Exception;
+
+ /**
+ * Gets the number of key/value pairs currently stored in the state. Note that is a key
+ * has been associated with "null", the key is removed from the state an will not
+ * be counted here.
+ *
+ * @return The number of key/value pairs currently stored in the state.
+ */
+ int size();
+
+ /**
+ * Disposes the key/value state, releasing all occupied resources.
+ */
+ void dispose();
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/KvStateSnapshot.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/KvStateSnapshot.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/KvStateSnapshot.java
new file mode 100644
index 0000000..6aa7a1e
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/KvStateSnapshot.java
@@ -0,0 +1,69 @@
+/*
+ * 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.state;
+
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+
+/**
+ * This class represents a snapshot of the {@link KvState}, taken for a checkpoint. Where exactly
+ * the snapshot stores the snapshot data (in this object, in an external data store, etc) depends
+ * on the actual implementation. This snapshot defines merely how to restore the state and
+ * how to discard the state.
+ *
+ * <p>One possible implementation is that this snapshot simply contains a copy of the key/value map.
+ *
+ * <p>Another possible implementation for this snapshot is that the key/value map is serialized into
+ * a file and this snapshot object contains a pointer to that file.
+ *
+ * @param <K> The type of the key
+ * @param <V> The type of the value
+ * @param <Backend> The type of the backend that can restore the state from this snapshot.
+ */
+public interface KvStateSnapshot<K, V, Backend extends StateBackend<Backend>> extends java.io.Serializable {
+
+ /**
+ * Loads the key/value state back from this snapshot.
+ *
+ *
+ * @param stateBackend The state backend that created this snapshot and can restore the key/value state
+ * from this snapshot.
+ * @param keySerializer The serializer for the keys.
+ * @param valueSerializer The serializer for the values.
+ * @param defaultValue The value that is returned when no other value has been associated with a key, yet.
+ * @param classLoader The class loader for user-defined types.
+ *
+ * @return An instance of the key/value state loaded from this snapshot.
+ *
+ * @throws Exception Exceptions can occur during the state loading and are forwarded.
+ */
+ KvState<K, V, Backend> restoreState(
+ Backend stateBackend,
+ TypeSerializer<K> keySerializer,
+ TypeSerializer<V> valueSerializer,
+ V defaultValue,
+ ClassLoader classLoader) throws Exception;
+
+
+ /**
+ * Discards the state snapshot, removing any resources occupied by it.
+ *
+ * @throws Exception Exceptions occurring during the state disposal should be forwarded.
+ */
+ void discardState() throws Exception;
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/OperatorStateHandle.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/OperatorStateHandle.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/OperatorStateHandle.java
deleted file mode 100644
index 0c0b2c9..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/OperatorStateHandle.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * 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.state;
-
-import java.io.Serializable;
-
-import org.apache.flink.runtime.state.StateHandle;
-
-public class OperatorStateHandle implements StateHandle<Serializable> {
-
- private static final long serialVersionUID = 1L;
-
- private final StateHandle<Serializable> handle;
- private final boolean isPartitioned;
-
- public OperatorStateHandle(StateHandle<Serializable> handle, boolean isPartitioned){
- this.handle = handle;
- this.isPartitioned = isPartitioned;
- }
-
- public boolean isPartitioned(){
- return isPartitioned;
- }
-
- @Override
- public Serializable getState(ClassLoader userCodeClassLoader) throws Exception {
- return handle.getState(userCodeClassLoader);
- }
-
- @Override
- public void discardState() throws Exception {
- handle.discardState();
- }
-
- public StateHandle<Serializable> getHandle() {
- return handle;
- }
-
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/PartitionedStateStore.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/PartitionedStateStore.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/PartitionedStateStore.java
deleted file mode 100644
index 34bfde7..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/PartitionedStateStore.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * 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.state;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.Map;
-
-import org.apache.flink.api.common.state.StateCheckpointer;
-import org.apache.flink.runtime.state.StateHandle;
-
-/**
- * Interface for storing and accessing partitioned state. The interface is
- * designed in a way that allows implementations for lazily state access.
- *
- * @param <S>
- * Type of the state.
- * @param <C>
- * Type of the state snapshot.
- */
-public interface PartitionedStateStore<S, C extends Serializable> {
-
- S getStateForKey(Serializable key) throws IOException;
-
- void setStateForKey(Serializable key, S state);
-
- void removeStateForKey(Serializable key);
-
- Map<Serializable, S> getPartitionedState() throws IOException;
-
- StateHandle<Serializable> snapshotStates(long checkpointId, long checkpointTimestamp) throws IOException;
-
- void restoreStates(StateHandle<Serializable> snapshot, ClassLoader userCodeClassLoader) throws Exception;
-
- boolean containsKey(Serializable key);
-
- void setCheckPointer(StateCheckpointer<S, C> checkpointer);
-
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/PartitionedStreamOperatorState.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/PartitionedStreamOperatorState.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/PartitionedStreamOperatorState.java
deleted file mode 100644
index 408a0f0..0000000
--- a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/PartitionedStreamOperatorState.java
+++ /dev/null
@@ -1,182 +0,0 @@
-/*
- * 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.state;
-
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.Map;
-
-import org.apache.flink.api.common.state.OperatorState;
-import org.apache.flink.api.common.state.StateCheckpointer;
-import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.runtime.state.StateHandle;
-import org.apache.flink.runtime.state.StateHandleProvider;
-import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
-import org.apache.flink.util.InstantiationUtil;
-
-/**
- * Implementation of the {@link OperatorState} interface for partitioned user
- * states. It provides methods for checkpointing and restoring partitioned
- * operator states upon failure.
- *
- * @param <IN>
- * Input type of the underlying {@link OneInputStreamOperator}
- * @param <S>
- * Type of the underlying {@link OperatorState}.
- * @param <C>
- * Type of the state snapshot.
- */
-public class PartitionedStreamOperatorState<IN, S, C extends Serializable> extends StreamOperatorState<S, C> {
-
- // KeySelector for getting the state partition key for each input
- private final KeySelector<IN, Serializable> keySelector;
-
- private final PartitionedStateStore<S, C> stateStore;
-
- private byte[] defaultState;
-
- // The currently processed input, used to extract the appropriate key
- private IN currentInput;
-
- private ClassLoader cl;
- private boolean restored = true;
- private StateHandle<Serializable> checkpoint = null;
-
- public PartitionedStreamOperatorState(StateCheckpointer<S, C> checkpointer,
- StateHandleProvider<C> provider, KeySelector<IN, Serializable> keySelector, ClassLoader cl) {
- super(checkpointer, provider);
- this.keySelector = keySelector;
- this.stateStore = new EagerStateStore<S, C>(checkpointer, provider);
- this.cl = cl;
- }
-
- @SuppressWarnings("unchecked")
- public PartitionedStreamOperatorState(StateHandleProvider<C> provider,
- KeySelector<IN, Serializable> keySelector, ClassLoader cl) {
- this((StateCheckpointer<S, C>) new BasicCheckpointer(), provider, keySelector, cl);
- }
-
- @SuppressWarnings("unchecked")
- @Override
- public S value() throws IOException {
- if (currentInput == null) {
- throw new IllegalStateException("Need a valid input for accessing the state.");
- } else {
- if (!restored) {
- // If the state is not restored yet, restore now
- restoreWithCheckpointer();
- }
- Serializable key;
- try {
- key = keySelector.getKey(currentInput);
- } catch (Exception e) {
- throw new RuntimeException("User-defined key selector threw an exception.", e);
- }
- if (stateStore.containsKey(key)) {
- return stateStore.getStateForKey(key);
- } else {
- try {
- return (S) checkpointer.restoreState((C) InstantiationUtil.deserializeObject(
- defaultState, cl));
- } catch (ClassNotFoundException e) {
- throw new RuntimeException("Could not deserialize default state value.", e);
- }
- }
- }
- }
-
- @Override
- public void update(S state) throws IOException {
- if (currentInput == null) {
- throw new IllegalStateException("Need a valid input for updating a state.");
- } else {
- if (!restored) {
- // If the state is not restored yet, restore now
- restoreWithCheckpointer();
- }
- Serializable key;
- try {
- key = keySelector.getKey(currentInput);
- } catch (Exception e) {
- throw new RuntimeException("User-defined key selector threw an exception.");
- }
-
- if (state == null) {
- // Remove state if set to null
- stateStore.removeStateForKey(key);
- } else {
- stateStore.setStateForKey(key, state);
- }
- }
- }
-
- @Override
- public void setDefaultState(S defaultState) {
- try {
- this.defaultState = InstantiationUtil.serializeObject(checkpointer.snapshotState(defaultState, 0, 0));
- } catch (IOException e) {
- throw new RuntimeException("Default state must be serializable.");
- }
- }
-
- public void setCurrentInput(IN input) {
- currentInput = input;
- }
-
- @Override
- public StateHandle<Serializable> snapshotState(long checkpointId, long checkpointTimestamp) throws Exception {
- // If the state is restored we take a snapshot, otherwise return the last checkpoint
- return restored ? stateStore.snapshotStates(checkpointId, checkpointTimestamp) : provider
- .createStateHandle(checkpoint.getState(cl));
- }
-
- @Override
- public void restoreState(StateHandle<Serializable> snapshot, ClassLoader userCodeClassLoader) throws Exception {
- // We store the snapshot for lazy restore
- checkpoint = snapshot;
- restored = false;
- }
-
- private void restoreWithCheckpointer() throws IOException {
- try {
- stateStore.restoreStates(checkpoint, cl);
- } catch (Exception e) {
- throw new IOException(e);
- }
- restored = true;
- checkpoint = null;
- }
-
- @Override
- public Map<Serializable, S> getPartitionedState() throws Exception {
- return stateStore.getPartitionedState();
- }
-
- @Override
- public void setCheckpointer(StateCheckpointer<S, C> checkpointer) {
- super.setCheckpointer(checkpointer);
- stateStore.setCheckPointer(checkpointer);
- }
-
- @Override
- public String toString() {
- return stateStore.toString();
- }
-
-}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StateBackend.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StateBackend.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StateBackend.java
new file mode 100644
index 0000000..b4fce7e
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StateBackend.java
@@ -0,0 +1,135 @@
+/*
+ * 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.state;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.runtime.state.StateHandle;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.Serializable;
+
+/**
+ * A state backend defines how state is stored and snapshotted during checkpoints.
+ *
+ * @param <Backend> The type of backend itself. This generic parameter is used to refer to the
+ * type of backend when creating state backed by this backend.
+ */
+public abstract class StateBackend<Backend extends StateBackend<Backend>> implements java.io.Serializable {
+
+ private static final long serialVersionUID = 4620413814639220247L;
+
+ // ------------------------------------------------------------------------
+ // initialization and cleanup
+ // ------------------------------------------------------------------------
+
+ /**
+ * This method is called by the task upon deployment to initialize the state backend for
+ * data for a specific job.
+ *
+ * @param job The ID of the job for which the state backend instance checkpoints data.
+ * @throws Exception Overwritten versions of this method may throw exceptions, in which
+ * case the job that uses the state backend is considered failed during
+ * deployment.
+ */
+ public abstract void initializeForJob(JobID job) throws Exception;
+
+ /**
+ * Disposes all state associated with the current job.
+ *
+ * @throws Exception Exceptions may occur during disposal of the state and should be forwarded.
+ */
+ public abstract void disposeAllStateForCurrentJob() throws Exception;
+
+ // ------------------------------------------------------------------------
+ // key/value state
+ // ------------------------------------------------------------------------
+
+ /**
+ * Creates a key/value state backed by this state backend.
+ *
+ * @param keySerializer The serializer for the key.
+ * @param valueSerializer The serializer for the value.
+ * @param defaultValue The value that is returned when no other value has been associated with a key, yet.
+ * @param <K> The type of the key.
+ * @param <V> The type of the value.
+ *
+ * @return A new key/value state backed by this backend.
+ *
+ * @throws Exception Exceptions may occur during initialization of the state and should be forwarded.
+ */
+ public abstract <K, V> KvState<K, V, Backend> createKvState(
+ TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer,
+ V defaultValue) throws Exception;
+
+
+ // ------------------------------------------------------------------------
+ // storing state for a checkpoint
+ // ------------------------------------------------------------------------
+
+ /**
+ * Creates an output stream that writes into the state of the given checkpoint. When the stream
+ * is closes, it returns a state handle that can retrieve the state back.
+ *
+ * @param checkpointID The ID of the checkpoint.
+ * @param timestamp The timestamp of the checkpoint.
+ * @return An output stream that writes state for the given checkpoint.
+ *
+ * @throws Exception Exceptions may occur while creating the stream and should be forwarded.
+ */
+ public abstract CheckpointStateOutputStream createCheckpointStateOutputStream(
+ long checkpointID, long timestamp) throws Exception;
+
+
+ /**
+ * Writes the given state into the checkpoint, and returns a handle that can retrieve the state back.
+ *
+ * @param state The state to be checkpointed.
+ * @param checkpointID The ID of the checkpoint.
+ * @param timestamp The timestamp of the checkpoint.
+ * @param <S> The type of the state.
+ *
+ * @return A state handle that can retrieve the checkpoined state.
+ *
+ * @throws Exception Exceptions may occur during serialization / storing the state and should be forwarded.
+ */
+ public abstract <S extends Serializable> StateHandle<S> checkpointStateSerializable(
+ S state, long checkpointID, long timestamp) throws Exception;
+
+
+ // ------------------------------------------------------------------------
+ // Checkpoint state output stream
+ // ------------------------------------------------------------------------
+
+ /**
+ * A dedicated output stream that produces a {@link StreamStateHandle} when closed.
+ */
+ public static abstract class CheckpointStateOutputStream extends OutputStream {
+
+ /**
+ * Closes the stream and gets a state handle that can create an input stream
+ * producing the data written to this stream.
+ *
+ * @return A state handle that can create an input stream producing the data written to this stream.
+ * @throws IOException Thrown, if the stream cannot be closed.
+ */
+ public abstract StreamStateHandle closeAndGetHandle() throws IOException;
+ }
+}
http://git-wip-us.apache.org/repos/asf/flink/blob/479bec0b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StateBackendFactory.java
----------------------------------------------------------------------
diff --git a/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StateBackendFactory.java b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StateBackendFactory.java
new file mode 100644
index 0000000..ad87eae
--- /dev/null
+++ b/flink-staging/flink-streaming/flink-streaming-core/src/main/java/org/apache/flink/streaming/api/state/StateBackendFactory.java
@@ -0,0 +1,40 @@
+/*
+ * 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.state;
+
+import org.apache.flink.configuration.Configuration;
+
+/**
+ * A factory to create a specific state backend. The state backend creation gets a Configuration
+ * object that can be used to read further config values.
+ *
+ * @param <T> The type of the state backend created.
+ */
+public interface StateBackendFactory<T extends StateBackend<T>> {
+
+ /**
+ * Creates the state backend, optionally using the given configuration.
+ *
+ * @param config The Flink configuration (loaded by the TaskManager).
+ * @return The created state backend.
+ *
+ * @throws Exception Exceptions during instantiation can be forwarded.
+ */
+ StateBackend<T> createFromConfig(Configuration config) throws Exception;
+}