Java tutorial
/* * 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.commons.lang3.StringUtils; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.operators.util.UserCodeObjectWrapper; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.io.network.partition.ResultPartitionType; import org.apache.flink.runtime.jobgraph.DistributionPattern; import org.apache.flink.runtime.jobgraph.InputFormatVertex; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.jobgraph.JobVertex; import org.apache.flink.runtime.jobgraph.JobVertexID; import org.apache.flink.runtime.jobgraph.ScheduleMode; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; import org.apache.flink.runtime.jobgraph.tasks.JobSnapshottingSettings; import org.apache.flink.runtime.jobmanager.scheduler.CoLocationGroup; import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup; import org.apache.flink.runtime.operators.util.TaskConfig; import org.apache.flink.streaming.api.CheckpointingMode; import org.apache.flink.streaming.api.environment.CheckpointConfig; 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.apache.flink.streaming.runtime.partitioner.RescalePartitioner; import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner; import org.apache.flink.streaming.runtime.tasks.StreamIterationHead; import org.apache.flink.streaming.runtime.tasks.StreamIterationTail; 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.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Queue; import java.util.Set; import static org.apache.flink.util.StringUtils.byteToHexString; @Internal public class StreamingJobGraphGenerator { private static final Logger LOG = LoggerFactory.getLogger(StreamingJobGraphGenerator.class); /** * Restart delay used for the FixedDelayRestartStrategy in case checkpointing was enabled but * no restart strategy has been specified. */ private static final long DEFAULT_RESTART_DELAY = 10000L; private StreamGraph streamGraph; private Map<Integer, JobVertex> jobVertices; private JobGraph jobGraph; private Collection<Integer> builtVertices; private List<StreamEdge> physicalEdgesInOrder; private Map<Integer, Map<Integer, StreamConfig>> chainedConfigs; private Map<Integer, StreamConfig> vertexConfigs; private Map<Integer, String> chainedNames; public StreamingJobGraphGenerator(StreamGraph streamGraph) { this.streamGraph = streamGraph; } private void init() { this.jobVertices = new HashMap<>(); this.builtVertices = new HashSet<>(); this.chainedConfigs = new HashMap<>(); this.vertexConfigs = new HashMap<>(); this.chainedNames = new HashMap<>(); this.physicalEdgesInOrder = new ArrayList<>(); } public JobGraph createJobGraph() { jobGraph = new JobGraph(streamGraph.getJobName()); // make sure that all vertices start immediately jobGraph.setScheduleMode(ScheduleMode.EAGER); init(); // Generate deterministic hashes for the nodes in order to identify them across // submission iff they didn't change. Map<Integer, byte[]> hashes = traverseStreamGraphAndGenerateHashes(); setChaining(hashes); setPhysicalEdges(); setSlotSharing(); configureCheckpointing(); // set the ExecutionConfig last when it has been finalized jobGraph.setExecutionConfig(streamGraph.getExecutionConfig()); return jobGraph; } private void setPhysicalEdges() { Map<Integer, List<StreamEdge>> physicalInEdgesInOrder = new HashMap<Integer, List<StreamEdge>>(); for (StreamEdge edge : physicalEdgesInOrder) { int target = edge.getTargetId(); List<StreamEdge> inEdges = physicalInEdgesInOrder.get(target); // create if not set if (inEdges == null) { inEdges = new ArrayList<>(); physicalInEdgesInOrder.put(target, inEdges); } inEdges.add(edge); } for (Map.Entry<Integer, List<StreamEdge>> inEdges : physicalInEdgesInOrder.entrySet()) { int vertex = inEdges.getKey(); List<StreamEdge> edgeList = inEdges.getValue(); vertexConfigs.get(vertex).setInPhysicalEdges(edgeList); } } /** * Sets up task chains from the source {@link StreamNode} instances. * * <p>This will recursively create all {@link JobVertex} instances. */ private void setChaining(Map<Integer, byte[]> hashes) { for (Integer sourceNodeId : streamGraph.getSourceIDs()) { createChain(sourceNodeId, sourceNodeId, hashes, 0); } } private List<StreamEdge> createChain(Integer startNodeId, Integer currentNodeId, Map<Integer, byte[]> hashes, int chainIndex) { if (!builtVertices.contains(startNodeId)) { List<StreamEdge> transitiveOutEdges = new ArrayList<StreamEdge>(); List<StreamEdge> chainableOutputs = new ArrayList<StreamEdge>(); List<StreamEdge> nonChainableOutputs = new ArrayList<StreamEdge>(); for (StreamEdge outEdge : streamGraph.getStreamNode(currentNodeId).getOutEdges()) { if (isChainable(outEdge)) { chainableOutputs.add(outEdge); } else { nonChainableOutputs.add(outEdge); } } for (StreamEdge chainable : chainableOutputs) { transitiveOutEdges .addAll(createChain(startNodeId, chainable.getTargetId(), hashes, chainIndex + 1)); } for (StreamEdge nonChainable : nonChainableOutputs) { transitiveOutEdges.add(nonChainable); createChain(nonChainable.getTargetId(), nonChainable.getTargetId(), hashes, 0); } chainedNames.put(currentNodeId, createChainedName(currentNodeId, chainableOutputs)); StreamConfig config = currentNodeId.equals(startNodeId) ? createJobVertex(startNodeId, hashes) : new StreamConfig(new Configuration()); setVertexConfig(currentNodeId, config, chainableOutputs, nonChainableOutputs); if (currentNodeId.equals(startNodeId)) { config.setChainStart(); config.setChainIndex(0); config.setOutEdgesInOrder(transitiveOutEdges); config.setOutEdges(streamGraph.getStreamNode(currentNodeId).getOutEdges()); for (StreamEdge edge : transitiveOutEdges) { connect(startNodeId, edge); } config.setTransitiveChainedTaskConfigs(chainedConfigs.get(startNodeId)); } else { Map<Integer, StreamConfig> chainedConfs = chainedConfigs.get(startNodeId); if (chainedConfs == null) { chainedConfigs.put(startNodeId, new HashMap<Integer, StreamConfig>()); } config.setChainIndex(chainIndex); chainedConfigs.get(startNodeId).put(currentNodeId, config); } return transitiveOutEdges; } else { return new ArrayList<>(); } } private String createChainedName(Integer vertexID, List<StreamEdge> chainedOutputs) { String operatorName = streamGraph.getStreamNode(vertexID).getOperatorName(); if (chainedOutputs.size() > 1) { List<String> outputChainedNames = new ArrayList<>(); for (StreamEdge chainable : chainedOutputs) { outputChainedNames.add(chainedNames.get(chainable.getTargetId())); } return operatorName + " -> (" + StringUtils.join(outputChainedNames, ", ") + ")"; } else if (chainedOutputs.size() == 1) { return operatorName + " -> " + chainedNames.get(chainedOutputs.get(0).getTargetId()); } else { return operatorName; } } private StreamConfig createJobVertex(Integer streamNodeId, Map<Integer, byte[]> hashes) { JobVertex jobVertex; StreamNode streamNode = streamGraph.getStreamNode(streamNodeId); byte[] hash = hashes.get(streamNodeId); if (hash == null) { throw new IllegalStateException( "Cannot find node hash. " + "Did you generate them before calling this method?"); } JobVertexID jobVertexId = new JobVertexID(hash); if (streamNode.getInputFormat() != null) { jobVertex = new InputFormatVertex(chainedNames.get(streamNodeId), jobVertexId); TaskConfig taskConfig = new TaskConfig(jobVertex.getConfiguration()); taskConfig.setStubWrapper(new UserCodeObjectWrapper<Object>(streamNode.getInputFormat())); } else { jobVertex = new JobVertex(chainedNames.get(streamNodeId), jobVertexId); } jobVertex.setInvokableClass(streamNode.getJobVertexClass()); int parallelism = streamNode.getParallelism(); if (parallelism > 0) { jobVertex.setParallelism(parallelism); } else { parallelism = jobVertex.getParallelism(); } int maxParallelism = streamNode.getMaxParallelism(); // the maximum parallelism specifies the upper bound for the parallelism if (parallelism > maxParallelism) { // the parallelism should always be smaller or equal than the max parallelism throw new IllegalStateException("The maximum parallelism (" + maxParallelism + ") of " + "the stream node " + streamNode + " is smaller than the parallelism (" + parallelism + "). Increase the maximum parallelism or decrease the parallelism of" + "this operator."); } else { jobVertex.setMaxParallelism(streamNode.getMaxParallelism()); } if (LOG.isDebugEnabled()) { LOG.debug("Parallelism set: {} for {}", parallelism, streamNodeId); } jobVertices.put(streamNodeId, jobVertex); builtVertices.add(streamNodeId); jobGraph.addVertex(jobVertex); return new StreamConfig(jobVertex.getConfiguration()); } @SuppressWarnings("unchecked") private void setVertexConfig(Integer vertexID, StreamConfig config, List<StreamEdge> chainableOutputs, List<StreamEdge> nonChainableOutputs) { StreamNode vertex = streamGraph.getStreamNode(vertexID); config.setVertexID(vertexID); config.setBufferTimeout(vertex.getBufferTimeout()); config.setTypeSerializerIn1(vertex.getTypeSerializerIn1()); config.setTypeSerializerIn2(vertex.getTypeSerializerIn2()); config.setTypeSerializerOut(vertex.getTypeSerializerOut()); config.setStreamOperator(vertex.getOperator()); config.setOutputSelectors(vertex.getOutputSelectors()); config.setNumberOfOutputs(nonChainableOutputs.size()); config.setNonChainedOutputs(nonChainableOutputs); config.setChainedOutputs(chainableOutputs); config.setTimeCharacteristic(streamGraph.getEnvironment().getStreamTimeCharacteristic()); final CheckpointConfig ceckpointCfg = streamGraph.getCheckpointConfig(); config.setStateBackend(streamGraph.getStateBackend()); config.setCheckpointingEnabled(ceckpointCfg.isCheckpointingEnabled()); if (ceckpointCfg.isCheckpointingEnabled()) { config.setCheckpointMode(ceckpointCfg.getCheckpointingMode()); } 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(0, vertex.getStatePartitioner1()); config.setStatePartitioner(1, vertex.getStatePartitioner2()); config.setStateKeySerializer(vertex.getStateKeySerializer()); // only set the max parallelism if the vertex uses partitioned state (= KeyedStream). if (vertex.getStatePartitioner1() != null) { config.setNumberOfKeyGroups(vertex.getMaxParallelism()); } Class<? extends AbstractInvokable> vertexClass = vertex.getJobVertexClass(); if (vertexClass.equals(StreamIterationHead.class) || vertexClass.equals(StreamIterationTail.class)) { config.setIterationId(streamGraph.getBrokerID(vertexID)); config.setIterationWaitTime(streamGraph.getLoopTimeout(vertexID)); } List<StreamEdge> allOutputs = new ArrayList<StreamEdge>(chainableOutputs); allOutputs.addAll(nonChainableOutputs); vertexConfigs.put(vertexID, config); } private void connect(Integer headOfChain, StreamEdge edge) { physicalEdgesInOrder.add(edge); Integer downStreamvertexID = edge.getTargetId(); JobVertex headVertex = jobVertices.get(headOfChain); JobVertex downStreamVertex = jobVertices.get(downStreamvertexID); StreamConfig downStreamConfig = new StreamConfig(downStreamVertex.getConfiguration()); downStreamConfig.setNumberOfInputs(downStreamConfig.getNumberOfInputs() + 1); StreamPartitioner<?> partitioner = edge.getPartitioner(); if (partitioner instanceof ForwardPartitioner) { downStreamVertex.connectNewDataSetAsInput(headVertex, DistributionPattern.POINTWISE, ResultPartitionType.PIPELINED, true); } else if (partitioner instanceof RescalePartitioner) { downStreamVertex.connectNewDataSetAsInput(headVertex, DistributionPattern.POINTWISE, ResultPartitionType.PIPELINED, true); } else { downStreamVertex.connectNewDataSetAsInput(headVertex, DistributionPattern.ALL_TO_ALL, ResultPartitionType.PIPELINED, true); } if (LOG.isDebugEnabled()) { LOG.debug("CONNECTED: {} - {} -> {}", partitioner.getClass().getSimpleName(), headOfChain, downStreamvertexID); } } private boolean isChainable(StreamEdge edge) { StreamNode upStreamVertex = edge.getSourceVertex(); StreamNode downStreamVertex = edge.getTargetVertex(); StreamOperator<?> headOperator = upStreamVertex.getOperator(); StreamOperator<?> outOperator = downStreamVertex.getOperator(); return downStreamVertex.getInEdges().size() == 1 && outOperator != null && headOperator != null && upStreamVertex.isSameSlotSharingGroup(downStreamVertex) && outOperator.getChainingStrategy() == ChainingStrategy.ALWAYS && (headOperator.getChainingStrategy() == ChainingStrategy.HEAD || headOperator.getChainingStrategy() == ChainingStrategy.ALWAYS) && (edge.getPartitioner() instanceof ForwardPartitioner) && upStreamVertex.getParallelism() == downStreamVertex.getParallelism() && streamGraph.isChainingEnabled(); } private void setSlotSharing() { Map<String, SlotSharingGroup> slotSharingGroups = new HashMap<>(); for (Entry<Integer, JobVertex> entry : jobVertices.entrySet()) { String slotSharingGroup = streamGraph.getStreamNode(entry.getKey()).getSlotSharingGroup(); SlotSharingGroup group = slotSharingGroups.get(slotSharingGroup); if (group == null) { group = new SlotSharingGroup(); slotSharingGroups.put(slotSharingGroup, group); } entry.getValue().setSlotSharingGroup(group); } for (Tuple2<StreamNode, StreamNode> pair : streamGraph.getIterationSourceSinkPairs()) { CoLocationGroup ccg = new CoLocationGroup(); JobVertex source = jobVertices.get(pair.f0.getId()); JobVertex sink = jobVertices.get(pair.f1.getId()); ccg.addVertex(source); ccg.addVertex(sink); source.updateCoLocationGroup(ccg); sink.updateCoLocationGroup(ccg); } } private void configureCheckpointing() { CheckpointConfig cfg = streamGraph.getCheckpointConfig(); if (cfg.isCheckpointingEnabled()) { long interval = cfg.getCheckpointInterval(); if (interval < 1) { throw new IllegalArgumentException("The checkpoint interval must be positive"); } // collect the vertices that receive "trigger checkpoint" messages. // currently, these are all the sources List<JobVertexID> triggerVertices = new ArrayList<>(); // collect the vertices that need to acknowledge the checkpoint // currently, these are all vertices List<JobVertexID> ackVertices = new ArrayList<>(jobVertices.size()); // collect the vertices that receive "commit checkpoint" messages // currently, these are all vertices List<JobVertexID> commitVertices = new ArrayList<>(); for (JobVertex vertex : jobVertices.values()) { if (vertex.isInputVertex()) { triggerVertices.add(vertex.getID()); } commitVertices.add(vertex.getID()); ackVertices.add(vertex.getID()); } JobSnapshottingSettings settings = new JobSnapshottingSettings(triggerVertices, ackVertices, commitVertices, interval, cfg.getCheckpointTimeout(), cfg.getMinPauseBetweenCheckpoints(), cfg.getMaxConcurrentCheckpoints()); jobGraph.setSnapshotSettings(settings); // check if a restart strategy has been set, if not then set the FixedDelayRestartStrategy if (streamGraph.getExecutionConfig().getRestartStrategy() == null) { // if the user enabled checkpointing, the default number of exec retries is infinite. streamGraph.getExecutionConfig().setRestartStrategy( RestartStrategies.fixedDelayRestart(Integer.MAX_VALUE, DEFAULT_RESTART_DELAY)); } } } // ------------------------------------------------------------------------ /** * 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. * * <p>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. * * <p>The generated hash is deterministic with respect to: * <ul> * <li>node-local properties (like parallelism, UDF, node ID), * <li>chained output nodes, and * <li>input nodes hashes * </ul> * * @return A map from {@link StreamNode#id} to hash as 16-byte array. */ private Map<Integer, byte[]> traverseStreamGraphAndGenerateHashes() { // The hash function used to generate the hash final HashFunction hashFunction = Hashing.murmur3_128(0); final Map<Integer, byte[]> hashes = new HashMap<>(); Set<Integer> visited = new HashSet<>(); Queue<StreamNode> 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<Integer> 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); } StreamNode currentNode; while ((currentNode = remaining.poll()) != null) { // Generate the hash code. Because multiple path exist to each // node, we might not have all required inputs available to // generate the hash code. if (generateNodeHash(currentNode, hashFunction, hashes)) { // Add the child nodes for (StreamEdge outEdge : currentNode.getOutEdges()) { StreamNode child = outEdge.getTargetVertex(); if (!visited.contains(child.getId())) { remaining.add(child); visited.add(child.getId()); } } } else { // We will revisit this later. visited.remove(currentNode.getId()); } } return hashes; } /** * Generates a hash for the node and returns whether the operation was * successful. * * @param node The node to generate the hash for * @param hashFunction The hash function to use * @param hashes The current state of generated hashes * @return <code>true</code> if the node hash has been generated. * <code>false</code>, otherwise. If the operation is not successful, the * hash needs be generated at a later point when all input is available. * @throws IllegalStateException If node has user-specified hash and is * intermediate node of a chain */ private boolean generateNodeHash(StreamNode node, HashFunction hashFunction, Map<Integer, byte[]> hashes) { // Check for user-specified ID String userSpecifiedHash = node.getTransformationId(); if (userSpecifiedHash == null) { // Check that all input nodes have their hashes computed for (StreamEdge inEdge : node.getInEdges()) { // If the input node has not been visited yet, the current // node will be visited again at a later point when all input // nodes have been visited and their hashes set. if (!hashes.containsKey(inEdge.getSourceId())) { return false; } } Hasher hasher = hashFunction.newHasher(); byte[] hash = generateDeterministicHash(node, hasher, hashes); if (hashes.put(node.getId(), hash) != null) { // Sanity check throw new IllegalStateException("Unexpected state. Tried to add node hash " + "twice. This is probably a bug in the JobGraph generator."); } return true; } else { // Check that this node is not part of a chain. This is currently // not supported, because the runtime takes the snapshots by the // operator ID of the first vertex in a chain. It's OK if the node // has chained outputs. for (StreamEdge inEdge : node.getInEdges()) { if (isChainable(inEdge)) { 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 " + node.getOperatorName() + "."); } } Hasher hasher = hashFunction.newHasher(); byte[] hash = generateUserSpecifiedHash(node, hasher); for (byte[] previousHash : hashes.values()) { if (Arrays.equals(previousHash, hash)) { throw new IllegalArgumentException("Hash collision on user-specified ID. " + "Most likely cause is a non-unique ID. Please check that all IDs " + "specified via `uid(String)` are unique."); } } if (hashes.put(node.getId(), hash) != null) { // Sanity check throw new IllegalStateException("Unexpected state. Tried to add node hash " + "twice. This is probably a bug in the JobGraph generator."); } return true; } } /** * Generates a hash from a user-specified ID. */ private byte[] generateUserSpecifiedHash(StreamNode node, Hasher hasher) { hasher.putString(node.getTransformationId(), Charset.forName("UTF-8")); return hasher.hash().asBytes(); } /** * Generates a deterministic hash from node-local properties and input and * output edges. */ private byte[] generateDeterministicHash(StreamNode node, Hasher hasher, Map<Integer, byte[]> hashes) { // Include stream node to hash. We use the current size of the computed // hashes as the ID. We cannot use the node's ID, because it is // assigned from a static counter. This will result in two identical // programs having different hashes. generateNodeLocalHash(node, hasher, hashes.size()); // Include chained nodes to hash for (StreamEdge outEdge : node.getOutEdges()) { if (isChainable(outEdge)) { StreamNode chainedNode = outEdge.getTargetVertex(); // Use the hash size again, because the nodes are chained to // this node. This does not add a hash for the chained nodes. generateNodeLocalHash(chainedNode, hasher, hashes.size()); } } byte[] hash = hasher.hash().asBytes(); // Make sure that all input nodes have their hash set before entering // this loop (calling this method). for (StreamEdge inEdge : node.getInEdges()) { byte[] otherHash = hashes.get(inEdge.getSourceId()); // Sanity check if (otherHash == null) { throw new IllegalStateException("Missing hash for input node " + inEdge.getSourceVertex() + ". Cannot generate hash for " + node + "."); } for (int j = 0; j < hash.length; j++) { hash[j] = (byte) (hash[j] * 37 ^ otherHash[j]); } } if (LOG.isDebugEnabled()) { String udfClassName = ""; if (node.getOperator() instanceof AbstractUdfStreamOperator) { udfClassName = ((AbstractUdfStreamOperator<?, ?>) node.getOperator()).getUserFunction().getClass() .getName(); } LOG.debug("Generated hash '" + byteToHexString(hash) + "' for node " + "'" + node.toString() + "' {id: " + node.getId() + ", " + "parallelism: " + node.getParallelism() + ", " + "user function: " + udfClassName + "}"); } return hash; } /** * Applies the {@link Hasher} to the {@link StreamNode} (only node local * attributes are taken into account). The hasher encapsulates the current * state of the hash. * * <p>The specified ID is local to this node. We cannot use the * {@link StreamNode#id}, because it is incremented in a static counter. * Therefore, the IDs for identical jobs will otherwise be different. */ private void generateNodeLocalHash(StreamNode node, Hasher hasher, int id) { // This resolves conflicts for otherwise identical source nodes. BUT // the generated hash codes depend on the ordering of the nodes in the // stream graph. hasher.putInt(id); if (node.getOperator() instanceof AbstractUdfStreamOperator) { String udfClassName = ((AbstractUdfStreamOperator<?, ?>) node.getOperator()).getUserFunction() .getClass().getName(); hasher.putString(udfClassName, Charset.forName("UTF-8")); } } }