New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[FLINK-5480] Introduce user-provided hash for JobVertexes #3117
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -18,6 +18,7 @@ | |
package org.apache.flink.streaming.connectors.cassandra; | ||
|
||
import com.datastax.driver.core.Cluster; | ||
import org.apache.flink.annotation.PublicEvolving; | ||
import org.apache.flink.api.common.typeinfo.TypeInformation; | ||
import org.apache.flink.api.common.typeutils.TypeSerializer; | ||
import org.apache.flink.api.java.tuple.Tuple; | ||
|
@@ -85,6 +86,7 @@ public CassandraSink<IN> name(String name) { | |
* @param uid The unique user-specified ID of this transformation. | ||
* @return The operator with the specified ID. | ||
*/ | ||
@PublicEvolving | ||
public CassandraSink<IN> uid(String uid) { | ||
if (useDataStreamSink) { | ||
getSinkTransformation().setUid(uid); | ||
|
@@ -94,6 +96,36 @@ public CassandraSink<IN> uid(String uid) { | |
return this; | ||
} | ||
|
||
/** | ||
* Sets an user provided hash for this operator. This will be used AS IS the create the JobVertexID. | ||
* <p/> | ||
* <p>The user provided hash is an alternative to the generated hashes, that is considered when identifying an | ||
* operator through the default hash mechanics fails (e.g. because of changes between Flink versions). | ||
* <p/> | ||
* <p><strong>Important</strong>: this should be used as a workaround or for trouble shooting. The provided hash | ||
* needs to be unique per transformation and job. Otherwise, job submission will fail. Furthermore, you cannot | ||
* assign user-specified hash to intermediate nodes in an operator chain and trying so will let your job fail. | ||
* | ||
* <p> | ||
* A use case for this is in migration between Flink versions or changing the jobs in a way that changes the | ||
* automatically generated hashes. In this case, providing the previous hashes directly through this method (e.g. | ||
* obtained from old logs) can help to reestablish a lost mapping from states to their target operator. | ||
* <p/> | ||
* | ||
* @param uidHash The user provided hash for this operator. This will become the JobVertexID, which is shown in the | ||
* logs and web ui. | ||
* @return The operator with the user provided hash. | ||
*/ | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We don't annotate classes outside of some projects, but should we add the |
||
@PublicEvolving | ||
public CassandraSink<IN> setUidHash(String uidHash) { | ||
if (useDataStreamSink) { | ||
getSinkTransformation().setUidHash(uidHash); | ||
} else { | ||
getStreamTransformation().setUidHash(uidHash); | ||
} | ||
return this; | ||
} | ||
|
||
/** | ||
* Sets the parallelism for this sink. The degree must be higher than zero. | ||
* | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,57 @@ | ||
/* | ||
* 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 org.apache.flink.util.StringUtils; | ||
|
||
import java.util.HashMap; | ||
import java.util.Map; | ||
|
||
/** | ||
* StreamGraphHasher that works with user provided hashes. This is useful in case we want to set (alternative) hashes | ||
* explicitly, e.g. to provide a way of manual backwards compatibility between versions when the mechanism of generating | ||
* hashes has changed in an incompatible way. | ||
* | ||
*/ | ||
public class StreamGraphUserHashHasher implements StreamGraphHasher { | ||
|
||
@Override | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. empty lines above missing |
||
public Map<Integer, byte[]> traverseStreamGraphAndGenerateHashes(StreamGraph streamGraph) { | ||
HashMap<Integer, byte[]> hashResult = new HashMap<>(); | ||
for (StreamNode streamNode : streamGraph.getStreamNodes()) { | ||
|
||
String userHash = streamNode.getUserHash(); | ||
|
||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. empty line |
||
if (null != userHash) { | ||
for (StreamEdge inEdge : streamNode.getInEdges()) { | ||
if (StreamingJobGraphGenerator.isChainable(inEdge, streamGraph)) { | ||
throw new UnsupportedOperationException("Cannot assign user-specified hash " | ||
+ "to intermediate node in chain. This will be supported in future " | ||
+ "versions of Flink. As a work around start new chain at task " | ||
+ streamNode.getOperatorName() + "."); | ||
} | ||
} | ||
|
||
hashResult.put(streamNode.getId(), StringUtils.hexStringToByte(userHash)); | ||
} | ||
} | ||
|
||
return hashResult; | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -69,7 +69,8 @@ public class StreamNode implements Serializable { | |
|
||
private InputFormat<?, ?> inputFormat; | ||
|
||
private String transformationId; | ||
private String transformationUID; | ||
private String userHash; | ||
|
||
public StreamNode(StreamExecutionEnvironment env, | ||
Integer id, | ||
|
@@ -272,12 +273,20 @@ public void setStateKeySerializer(TypeSerializer<?> stateKeySerializer) { | |
this.stateKeySerializer = stateKeySerializer; | ||
} | ||
|
||
public String getTransformationId() { | ||
return transformationId; | ||
public String getTransformationUID() { | ||
return transformationUID; | ||
} | ||
|
||
void setTransformationId(String transformationId) { | ||
this.transformationId = transformationId; | ||
void setTransformationUID(String transformationId) { | ||
this.transformationUID = transformationId; | ||
} | ||
|
||
public String getUserHash() { | ||
return userHash; | ||
} | ||
|
||
public void setUserHash(String userHash) { | ||
this.userHash = userHash; | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Since this is executed on the client before submitting the job it might be helpful to do some early sanity checking here. The expected String is a hex representation of a JobVertexID |
||
} | ||
|
||
@Override | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Not needed