[jira] [Commented] (FLINK-5290) Ensure backwards compatibility of the hashes used to generate JobVertexIds

2016-12-14 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/FLINK-5290?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15748075#comment-15748075
 ] 

ASF GitHub Bot commented on FLINK-5290:
---

Github user StefanRRichter closed the pull request at:

https://github.com/apache/flink/pull/2966


> Ensure backwards compatibility of the hashes used to generate JobVertexIds
> --
>
> Key: FLINK-5290
> URL: https://issues.apache.org/jira/browse/FLINK-5290
> Project: Flink
>  Issue Type: Sub-task
>  Components: State Backends, Checkpointing
>Reporter: Stefan Richter
>Assignee: Stefan Richter
>
> The way in which hashes for JobVertexIds are generated changed between Flink 
> 1.1 and 1.2 (parallelism was considered for the hash in 1.1). We need to be 
> backwards compatible to old JobVertexId generation so that we can still 
> assign state from old savepoints.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-5290) Ensure backwards compatibility of the hashes used to generate JobVertexIds

2016-12-14 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/FLINK-5290?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15748070#comment-15748070
 ] 

ASF GitHub Bot commented on FLINK-5290:
---

Github user aljoscha commented on the issue:

https://github.com/apache/flink/pull/2966
  
@StefanRRichter Thanks for you work! đź‘Ť 

I merged this, could you please close the Jira issue and this PR?


> Ensure backwards compatibility of the hashes used to generate JobVertexIds
> --
>
> Key: FLINK-5290
> URL: https://issues.apache.org/jira/browse/FLINK-5290
> Project: Flink
>  Issue Type: Sub-task
>  Components: State Backends, Checkpointing
>Reporter: Stefan Richter
>Assignee: Stefan Richter
>
> The way in which hashes for JobVertexIds are generated changed between Flink 
> 1.1 and 1.2 (parallelism was considered for the hash in 1.1). We need to be 
> backwards compatible to old JobVertexId generation so that we can still 
> assign state from old savepoints.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-5290) Ensure backwards compatibility of the hashes used to generate JobVertexIds

2016-12-13 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/FLINK-5290?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15745665#comment-15745665
 ] 

ASF GitHub Bot commented on FLINK-5290:
---

Github user StefanRRichter commented on a diff in the pull request:

https://github.com/apache/flink/pull/2966#discussion_r92219738
  
--- Diff: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphHasherV2.java
 ---
@@ -0,0 +1,319 @@
+/*
+ * 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.graph;
+
+import com.google.common.hash.HashFunction;
+import com.google.common.hash.Hasher;
+import com.google.common.hash.Hashing;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.transformations.StreamTransformation;
+import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.charset.Charset;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import static org.apache.flink.util.StringUtils.byteToHexString;
+
+/**
+ * StreamGraphHasher from Flink 1.2. This contains duplicated code to 
ensure that the algorithm does not change with
+ * future Flink versions.
+ * 
+ * DO NOT MODIFY THIS CLASS
+ */
+public class StreamGraphHasherV2 implements StreamGraphHasher {
+
+   private static final Logger LOG = 
LoggerFactory.getLogger(StreamGraphHasherV2.class);
+
+   /**
+* Returns a map with a hash for each {@link StreamNode} of the {@link
+* StreamGraph}. The hash is used as the {@link JobVertexID} in order to
+* identify nodes across job submissions if they didn't change.
+* 
--- End diff --

Thanks, I fixed those.


> Ensure backwards compatibility of the hashes used to generate JobVertexIds
> --
>
> Key: FLINK-5290
> URL: https://issues.apache.org/jira/browse/FLINK-5290
> Project: Flink
>  Issue Type: Sub-task
>  Components: State Backends, Checkpointing
>Reporter: Stefan Richter
>Assignee: Stefan Richter
>
> The way in which hashes for JobVertexIds are generated changed between Flink 
> 1.1 and 1.2 (parallelism was considered for the hash in 1.1). We need to be 
> backwards compatible to old JobVertexId generation so that we can still 
> assign state from old savepoints.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-5290) Ensure backwards compatibility of the hashes used to generate JobVertexIds

2016-12-13 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/FLINK-5290?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15745166#comment-15745166
 ] 

ASF GitHub Bot commented on FLINK-5290:
---

Github user aljoscha commented on a diff in the pull request:

https://github.com/apache/flink/pull/2966#discussion_r92167823
  
--- Diff: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphHasherV2.java
 ---
@@ -0,0 +1,319 @@
+/*
+ * 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.graph;
+
+import com.google.common.hash.HashFunction;
+import com.google.common.hash.Hasher;
+import com.google.common.hash.Hashing;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.transformations.StreamTransformation;
+import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.charset.Charset;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import static org.apache.flink.util.StringUtils.byteToHexString;
+
+/**
+ * StreamGraphHasher from Flink 1.2. This contains duplicated code to 
ensure that the algorithm does not change with
+ * future Flink versions.
+ * 
+ * DO NOT MODIFY THIS CLASS
+ */
+public class StreamGraphHasherV2 implements StreamGraphHasher {
+
+   private static final Logger LOG = 
LoggerFactory.getLogger(StreamGraphHasherV2.class);
+
+   /**
+* Returns a map with a hash for each {@link StreamNode} of the {@link
+* StreamGraph}. The hash is used as the {@link JobVertexID} in order to
+* identify nodes across job submissions if they didn't change.
+* 
--- End diff --

duplicate `` and no blank line. This appears several times in this PR.


> Ensure backwards compatibility of the hashes used to generate JobVertexIds
> --
>
> Key: FLINK-5290
> URL: https://issues.apache.org/jira/browse/FLINK-5290
> Project: Flink
>  Issue Type: Sub-task
>  Components: State Backends, Checkpointing
>Reporter: Stefan Richter
>Assignee: Stefan Richter
>
> The way in which hashes for JobVertexIds are generated changed between Flink 
> 1.1 and 1.2 (parallelism was considered for the hash in 1.1). We need to be 
> backwards compatible to old JobVertexId generation so that we can still 
> assign state from old savepoints.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-5290) Ensure backwards compatibility of the hashes used to generate JobVertexIds

2016-12-13 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/FLINK-5290?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15745165#comment-15745165
 ] 

ASF GitHub Bot commented on FLINK-5290:
---

Github user aljoscha commented on a diff in the pull request:

https://github.com/apache/flink/pull/2966#discussion_r92167706
  
--- Diff: 
flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/api/graph/StreamGraphHasherV1.java
 ---
@@ -0,0 +1,293 @@
+/*
+ * 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.migration.streaming.api.graph;
+
+import com.google.common.hash.HashFunction;
+import com.google.common.hash.Hasher;
+import com.google.common.hash.Hashing;
+import org.apache.flink.streaming.api.graph.StreamEdge;
+import org.apache.flink.streaming.api.graph.StreamGraph;
+import org.apache.flink.streaming.api.graph.StreamGraphHasher;
+import org.apache.flink.streaming.api.graph.StreamNode;
+import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.charset.Charset;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import static org.apache.flink.util.StringUtils.byteToHexString;
+
+/**
+ * StreamGraphHasher from Flink 1.1. This contains duplicated code to 
ensure that the algorithm does not change with
+ * future Flink versions.
+ * 
--- End diff --

This should be
```
...
future Flink versions.

DO NOT MODIFY THIS CLASS
```

i.e. a blank line is missing and the `` can be on the same line, doesn't 
have to be, though.


> Ensure backwards compatibility of the hashes used to generate JobVertexIds
> --
>
> Key: FLINK-5290
> URL: https://issues.apache.org/jira/browse/FLINK-5290
> Project: Flink
>  Issue Type: Sub-task
>  Components: State Backends, Checkpointing
>Reporter: Stefan Richter
>Assignee: Stefan Richter
>
> The way in which hashes for JobVertexIds are generated changed between Flink 
> 1.1 and 1.2 (parallelism was considered for the hash in 1.1). We need to be 
> backwards compatible to old JobVertexId generation so that we can still 
> assign state from old savepoints.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-5290) Ensure backwards compatibility of the hashes used to generate JobVertexIds

2016-12-13 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/FLINK-5290?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15744901#comment-15744901
 ] 

ASF GitHub Bot commented on FLINK-5290:
---

Github user StefanRRichter commented on the issue:

https://github.com/apache/flink/pull/2966
  
@aljoscha I duplicated the code in different hasher versions and moved the 
old version to a migration package.


> Ensure backwards compatibility of the hashes used to generate JobVertexIds
> --
>
> Key: FLINK-5290
> URL: https://issues.apache.org/jira/browse/FLINK-5290
> Project: Flink
>  Issue Type: Sub-task
>  Components: State Backends, Checkpointing
>Reporter: Stefan Richter
>Assignee: Stefan Richter
>
> The way in which hashes for JobVertexIds are generated changed between Flink 
> 1.1 and 1.2 (parallelism was considered for the hash in 1.1). We need to be 
> backwards compatible to old JobVertexId generation so that we can still 
> assign state from old savepoints.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-5290) Ensure backwards compatibility of the hashes used to generate JobVertexIds

2016-12-13 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/FLINK-5290?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15744689#comment-15744689
 ] 

ASF GitHub Bot commented on FLINK-5290:
---

Github user aljoscha commented on a diff in the pull request:

https://github.com/apache/flink/pull/2966#discussion_r92128811
  
--- Diff: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/DefaultStreamGraphHasher.java
 ---
@@ -0,0 +1,313 @@
+/*
+ * 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.graph;
+
+import com.google.common.hash.HashFunction;
+import com.google.common.hash.Hasher;
+import com.google.common.hash.Hashing;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.transformations.StreamTransformation;
+import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.charset.Charset;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import static org.apache.flink.util.StringUtils.byteToHexString;
+
+public class DefaultStreamGraphHasher implements StreamGraphHasher {
+
+   private static final Logger LOG = 
LoggerFactory.getLogger(DefaultStreamGraphHasher.class);
+
+   /**
+* Returns a map with a hash for each {@link StreamNode} of the {@link
+* StreamGraph}. The hash is used as the {@link JobVertexID} in order to
+* identify nodes across job submissions if they didn't change.
+* 
+* The complete {@link StreamGraph} is traversed. The hash is either
+* computed from the transformation's user-specified id (see
+* {@link StreamTransformation#getUid()}) or generated in a 
deterministic way.
+* 
+* The generated hash is deterministic with respect to:
+* 
+* node-local properties (like parallelism, UDF, node ID),
+* chained output nodes, and
+* input nodes hashes
+* 
+*
+* @return A map from {@link StreamNode#id} to hash as 16-byte array.
+*/
+   @Override
+   public Map 
traverseStreamGraphAndGenerateHashes(StreamGraph streamGraph) {
+   // The hash function used to generate the hash
+   final HashFunction hashFunction = Hashing.murmur3_128(0);
+   final Map hashes = new HashMap<>();
+
+   Set visited = new HashSet<>();
+   Queue remaining = new ArrayDeque<>();
+
+   // We need to make the source order deterministic. The source 
IDs are
+   // not returned in the same order, which means that submitting 
the same
+   // program twice might result in different traversal, which 
breaks the
+   // deterministic hash assignment.
+   List sources = new ArrayList<>();
+   for (Integer sourceNodeId : streamGraph.getSourceIDs()) {
+   sources.add(sourceNodeId);
+   }
+   Collections.sort(sources);
+
+   //
+   // Traverse the graph in a breadth-first manner. Keep in mind 
that
+   // the graph is not a tree and multiple paths to nodes can 
exist.
+   //
+
+   // Start with source nodes
+   for (Integer sourceNodeId : sources) {
+   remaining.add(streamGraph.getStreamNode(sourceNodeId));
+   visited.add(sourceNodeId);
+   }
+
+   

[jira] [Commented] (FLINK-5290) Ensure backwards compatibility of the hashes used to generate JobVertexIds

2016-12-12 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/FLINK-5290?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15742637#comment-15742637
 ] 

ASF GitHub Bot commented on FLINK-5290:
---

Github user StefanRRichter commented on a diff in the pull request:

https://github.com/apache/flink/pull/2966#discussion_r92004254
  
--- Diff: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/DefaultStreamGraphHasher.java
 ---
@@ -0,0 +1,313 @@
+/*
+ * 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.graph;
+
+import com.google.common.hash.HashFunction;
+import com.google.common.hash.Hasher;
+import com.google.common.hash.Hashing;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.transformations.StreamTransformation;
+import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.charset.Charset;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import static org.apache.flink.util.StringUtils.byteToHexString;
+
+public class DefaultStreamGraphHasher implements StreamGraphHasher {
+
+   private static final Logger LOG = 
LoggerFactory.getLogger(DefaultStreamGraphHasher.class);
+
+   /**
+* Returns a map with a hash for each {@link StreamNode} of the {@link
+* StreamGraph}. The hash is used as the {@link JobVertexID} in order to
+* identify nodes across job submissions if they didn't change.
+* 
+* The complete {@link StreamGraph} is traversed. The hash is either
+* computed from the transformation's user-specified id (see
+* {@link StreamTransformation#getUid()}) or generated in a 
deterministic way.
+* 
+* The generated hash is deterministic with respect to:
+* 
+* node-local properties (like parallelism, UDF, node ID),
+* chained output nodes, and
+* input nodes hashes
+* 
+*
+* @return A map from {@link StreamNode#id} to hash as 16-byte array.
+*/
+   @Override
+   public Map 
traverseStreamGraphAndGenerateHashes(StreamGraph streamGraph) {
+   // The hash function used to generate the hash
+   final HashFunction hashFunction = Hashing.murmur3_128(0);
+   final Map hashes = new HashMap<>();
+
+   Set visited = new HashSet<>();
+   Queue remaining = new ArrayDeque<>();
+
+   // We need to make the source order deterministic. The source 
IDs are
+   // not returned in the same order, which means that submitting 
the same
+   // program twice might result in different traversal, which 
breaks the
+   // deterministic hash assignment.
+   List sources = new ArrayList<>();
+   for (Integer sourceNodeId : streamGraph.getSourceIDs()) {
+   sources.add(sourceNodeId);
+   }
+   Collections.sort(sources);
+
+   //
+   // Traverse the graph in a breadth-first manner. Keep in mind 
that
+   // the graph is not a tree and multiple paths to nodes can 
exist.
+   //
+
+   // Start with source nodes
+   for (Integer sourceNodeId : sources) {
+   remaining.add(streamGraph.getStreamNode(sourceNodeId));
+   visited.add(sourceNodeId);
+   }
+
  

[jira] [Commented] (FLINK-5290) Ensure backwards compatibility of the hashes used to generate JobVertexIds

2016-12-12 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/FLINK-5290?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15742622#comment-15742622
 ] 

ASF GitHub Bot commented on FLINK-5290:
---

Github user aljoscha commented on a diff in the pull request:

https://github.com/apache/flink/pull/2966#discussion_r92003177
  
--- Diff: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/DefaultStreamGraphHasher.java
 ---
@@ -0,0 +1,313 @@
+/*
+ * 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.graph;
+
+import com.google.common.hash.HashFunction;
+import com.google.common.hash.Hasher;
+import com.google.common.hash.Hashing;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.transformations.StreamTransformation;
+import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.charset.Charset;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import static org.apache.flink.util.StringUtils.byteToHexString;
+
+public class DefaultStreamGraphHasher implements StreamGraphHasher {
+
+   private static final Logger LOG = 
LoggerFactory.getLogger(DefaultStreamGraphHasher.class);
+
+   /**
+* Returns a map with a hash for each {@link StreamNode} of the {@link
+* StreamGraph}. The hash is used as the {@link JobVertexID} in order to
+* identify nodes across job submissions if they didn't change.
+* 
+* The complete {@link StreamGraph} is traversed. The hash is either
+* computed from the transformation's user-specified id (see
+* {@link StreamTransformation#getUid()}) or generated in a 
deterministic way.
+* 
+* The generated hash is deterministic with respect to:
+* 
+* node-local properties (like parallelism, UDF, node ID),
+* chained output nodes, and
+* input nodes hashes
+* 
+*
+* @return A map from {@link StreamNode#id} to hash as 16-byte array.
+*/
+   @Override
+   public Map 
traverseStreamGraphAndGenerateHashes(StreamGraph streamGraph) {
+   // The hash function used to generate the hash
+   final HashFunction hashFunction = Hashing.murmur3_128(0);
+   final Map hashes = new HashMap<>();
+
+   Set visited = new HashSet<>();
+   Queue remaining = new ArrayDeque<>();
+
+   // We need to make the source order deterministic. The source 
IDs are
+   // not returned in the same order, which means that submitting 
the same
+   // program twice might result in different traversal, which 
breaks the
+   // deterministic hash assignment.
+   List sources = new ArrayList<>();
+   for (Integer sourceNodeId : streamGraph.getSourceIDs()) {
+   sources.add(sourceNodeId);
+   }
+   Collections.sort(sources);
+
+   //
+   // Traverse the graph in a breadth-first manner. Keep in mind 
that
+   // the graph is not a tree and multiple paths to nodes can 
exist.
+   //
+
+   // Start with source nodes
+   for (Integer sourceNodeId : sources) {
+   remaining.add(streamGraph.getStreamNode(sourceNodeId));
+   visited.add(sourceNodeId);
+   }
+
+   

[jira] [Commented] (FLINK-5290) Ensure backwards compatibility of the hashes used to generate JobVertexIds

2016-12-12 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/FLINK-5290?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15742541#comment-15742541
 ] 

ASF GitHub Bot commented on FLINK-5290:
---

Github user StefanRRichter commented on a diff in the pull request:

https://github.com/apache/flink/pull/2966#discussion_r91997425
  
--- Diff: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/DefaultStreamGraphHasher.java
 ---
@@ -0,0 +1,313 @@
+/*
+ * 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.graph;
+
+import com.google.common.hash.HashFunction;
+import com.google.common.hash.Hasher;
+import com.google.common.hash.Hashing;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+import org.apache.flink.streaming.api.operators.StreamOperator;
+import org.apache.flink.streaming.api.transformations.StreamTransformation;
+import org.apache.flink.streaming.runtime.partitioner.ForwardPartitioner;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.charset.Charset;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import static org.apache.flink.util.StringUtils.byteToHexString;
+
+public class DefaultStreamGraphHasher implements StreamGraphHasher {
+
+   private static final Logger LOG = 
LoggerFactory.getLogger(DefaultStreamGraphHasher.class);
+
+   /**
+* Returns a map with a hash for each {@link StreamNode} of the {@link
+* StreamGraph}. The hash is used as the {@link JobVertexID} in order to
+* identify nodes across job submissions if they didn't change.
+* 
+* The complete {@link StreamGraph} is traversed. The hash is either
+* computed from the transformation's user-specified id (see
+* {@link StreamTransformation#getUid()}) or generated in a 
deterministic way.
+* 
+* The generated hash is deterministic with respect to:
+* 
+* node-local properties (like parallelism, UDF, node ID),
+* chained output nodes, and
+* input nodes hashes
+* 
+*
+* @return A map from {@link StreamNode#id} to hash as 16-byte array.
+*/
+   @Override
+   public Map 
traverseStreamGraphAndGenerateHashes(StreamGraph streamGraph) {
+   // The hash function used to generate the hash
+   final HashFunction hashFunction = Hashing.murmur3_128(0);
+   final Map hashes = new HashMap<>();
+
+   Set visited = new HashSet<>();
+   Queue remaining = new ArrayDeque<>();
+
+   // We need to make the source order deterministic. The source 
IDs are
+   // not returned in the same order, which means that submitting 
the same
+   // program twice might result in different traversal, which 
breaks the
+   // deterministic hash assignment.
+   List sources = new ArrayList<>();
+   for (Integer sourceNodeId : streamGraph.getSourceIDs()) {
+   sources.add(sourceNodeId);
+   }
+   Collections.sort(sources);
+
+   //
+   // Traverse the graph in a breadth-first manner. Keep in mind 
that
+   // the graph is not a tree and multiple paths to nodes can 
exist.
+   //
+
+   // Start with source nodes
+   for (Integer sourceNodeId : sources) {
+   remaining.add(streamGraph.getStreamNode(sourceNodeId));
+   visited.add(sourceNodeId);
+   }
+
  

[jira] [Commented] (FLINK-5290) Ensure backwards compatibility of the hashes used to generate JobVertexIds

2016-12-08 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/FLINK-5290?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15732325#comment-15732325
 ] 

ASF GitHub Bot commented on FLINK-5290:
---

Github user uce commented on the issue:

https://github.com/apache/flink/pull/2966
  
Stephan, Aljoscha: I can take a look at this tomorrow or next week.


> Ensure backwards compatibility of the hashes used to generate JobVertexIds
> --
>
> Key: FLINK-5290
> URL: https://issues.apache.org/jira/browse/FLINK-5290
> Project: Flink
>  Issue Type: Sub-task
>  Components: State Backends, Checkpointing
>Reporter: Stefan Richter
>Assignee: Stefan Richter
>
> The way in which hashes for JobVertexIds are generated changed between Flink 
> 1.1 and 1.2 (parallelism was considered for the hash in 1.1). We need to be 
> backwards compatible to old JobVertexId generation so that we can still 
> assign state from old savepoints.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-5290) Ensure backwards compatibility of the hashes used to generate JobVertexIds

2016-12-08 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/FLINK-5290?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15732273#comment-15732273
 ] 

ASF GitHub Bot commented on FLINK-5290:
---

GitHub user StefanRRichter opened a pull request:

https://github.com/apache/flink/pull/2966

[FLINK-5290] Ensure backwards compatibility of the hashes used to gen…

The way in which hashes for JobVertexIds are generated changed between 
Flink 1.1 and 1.2 (parallelism was considered for the hash in 1.1). We need to 
be backwards compatible to old JobVertexId generation so that we can still 
assign state from old savepoints.

This PR introduced backwards compatibility for hashes, by considering 
alternative hashes through different ``StreamGraphHasher`` implementations.

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/StefanRRichter/flink 
job-vertex-hash-backwards-compatibility

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/flink/pull/2966.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #2966


commit dd84dd393be8f4c470b0087841bb4b681f67430b
Author: Stefan Richter 
Date:   2016-12-08T13:03:47Z

[FLINK-5290] Ensure backwards compatibility of the hashes used to generate 
JobVertexIds




> Ensure backwards compatibility of the hashes used to generate JobVertexIds
> --
>
> Key: FLINK-5290
> URL: https://issues.apache.org/jira/browse/FLINK-5290
> Project: Flink
>  Issue Type: Sub-task
>  Components: State Backends, Checkpointing
>Reporter: Stefan Richter
>Assignee: Stefan Richter
>
> The way in which hashes for JobVertexIds are generated changed between Flink 
> 1.1 and 1.2 (parallelism was considered for the hash in 1.1). We need to be 
> backwards compatible to old JobVertexId generation so that we can still 
> assign state from old savepoints.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (FLINK-5290) Ensure backwards compatibility of the hashes used to generate JobVertexIds

2016-12-08 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/FLINK-5290?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15732278#comment-15732278
 ] 

ASF GitHub Bot commented on FLINK-5290:
---

Github user StefanRRichter commented on the issue:

https://github.com/apache/flink/pull/2966
  
CC @aljoscha @uce @StephanEwen 


> Ensure backwards compatibility of the hashes used to generate JobVertexIds
> --
>
> Key: FLINK-5290
> URL: https://issues.apache.org/jira/browse/FLINK-5290
> Project: Flink
>  Issue Type: Sub-task
>  Components: State Backends, Checkpointing
>Reporter: Stefan Richter
>Assignee: Stefan Richter
>
> The way in which hashes for JobVertexIds are generated changed between Flink 
> 1.1 and 1.2 (parallelism was considered for the hash in 1.1). We need to be 
> backwards compatible to old JobVertexId generation so that we can still 
> assign state from old savepoints.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)