[docs]defcull(dsk,keys):"""Return new dask with only the tasks required to calculate keys. In other words, remove unnecessary tasks from dask. ``keys`` may be a single key or list of keys. Examples -------- >>> def inc(x): ... return x + 1 >>> def add(x, y): ... return x + y >>> d = {'x': 1, 'y': (inc, 'x'), 'out': (add, 'x', 10)} >>> dsk, dependencies = cull(d, 'out') >>> dsk # doctest: +ELLIPSIS {'out': (<function add at ...>, 'x', 10), 'x': 1} >>> dependencies # doctest: +ELLIPSIS {'out': ['x'], 'x': []} Returns ------- dsk: culled dask graph dependencies: Dict mapping {key: [deps]}. Useful side effect to accelerate other optimizations, notably fuse. """ifnotisinstance(keys,(list,set)):keys=[keys]seen=set()dependencies=dict()out={}work=list(set(flatten(keys)))whilework:new_work=[]forkinwork:dependencies_k=get_dependencies(dsk,k,as_list=True)# fuse needs listsout[k]=dsk[k]dependencies[k]=dependencies_kfordindependencies_k:ifdnotinseen:seen.add(d)new_work.append(d)work=new_workreturnout,dependencies
defdefault_fused_linear_keys_renamer(keys):"""Create new keys for fused tasks"""typ=type(keys[0])iftypisstr:names=[utils.key_split(x)forxinkeys[:0:-1]]names.append(keys[0])return"-".join(names)eliftypistupleandlen(keys[0])>0andisinstance(keys[0][0],str):names=[utils.key_split(x)forxinkeys[:0:-1]]names.append(keys[0][0])return("-".join(names),)+keys[0][1:]else:returnNonedeffuse_linear(dsk,keys=None,dependencies=None,rename_keys=True):"""Return new dask graph with linear sequence of tasks fused together. If specified, the keys in ``keys`` keyword argument are *not* fused. Supply ``dependencies`` from output of ``cull`` if available to avoid recomputing dependencies. **This function is mostly superseded by ``fuse``** Parameters ---------- dsk: dict keys: list dependencies: dict, optional {key: [list-of-keys]}. Must be a list to provide count of each key This optional input often comes from ``cull`` rename_keys: bool or func, optional Whether to rename fused keys with ``default_fused_linear_keys_renamer`` or not. Renaming fused keys can keep the graph more understandable and comprehensive, but it comes at the cost of additional processing. If False, then the top-most key will be used. For advanced usage, a func is also accepted, ``new_key = rename_keys(fused_key_list)``. Examples -------- >>> def inc(x): ... return x + 1 >>> def add(x, y): ... return x + y >>> d = {'a': 1, 'b': (inc, 'a'), 'c': (inc, 'b')} >>> dsk, dependencies = fuse(d) >>> dsk # doctest: +SKIP {'a-b-c': (inc, (inc, 1)), 'c': 'a-b-c'} >>> dsk, dependencies = fuse(d, rename_keys=False) >>> dsk # doctest: +ELLIPSIS {'c': (<function inc at ...>, (<function inc at ...>, 1))} >>> dsk, dependencies = fuse(d, keys=['b'], rename_keys=False) >>> dsk # doctest: +ELLIPSIS {'b': (<function inc at ...>, 1), 'c': (<function inc at ...>, 'b')} Returns ------- dsk: output graph with keys fused dependencies: dict mapping dependencies after fusion. Useful side effect to accelerate other downstream optimizations. """ifkeysisnotNoneandnotisinstance(keys,set):ifnotisinstance(keys,list):keys=[keys]keys=set(flatten(keys))ifdependenciesisNone:dependencies={k:get_dependencies(dsk,k,as_list=True)forkindsk}# locate all members of linear chainschild2parent={}unfusible=set()forparentindsk:deps=dependencies[parent]has_many_children=len(deps)>1forchildindeps:ifkeysisnotNoneandchildinkeys:unfusible.add(child)elifchildinchild2parent:delchild2parent[child]unfusible.add(child)elifhas_many_children:unfusible.add(child)elifchildnotinunfusible:child2parent[child]=parent# construct the chains from ancestor to descendantchains=[]parent2child=dict(map(reversed,child2parent.items()))whilechild2parent:child,parent=child2parent.popitem()chain=[child,parent]whileparentinchild2parent:parent=child2parent.pop(parent)delparent2child[parent]chain.append(parent)chain.reverse()whilechildinparent2child:child=parent2child.pop(child)delchild2parent[child]chain.append(child)chains.append(chain)dependencies={k:set(v)fork,vindependencies.items()}ifrename_keysisTrue:key_renamer=default_fused_linear_keys_renamerelifrename_keysisFalse:key_renamer=Noneelse:key_renamer=rename_keys# create a new dask with fused chainsrv={}fused=set()aliases=set()is_renamed=Falseforchaininchains:ifkey_renamerisnotNone:new_key=key_renamer(chain)is_renamed=(new_keyisnotNoneandnew_keynotindskandnew_keynotinrv)child=chain.pop()val=dsk[child]whilechain:parent=chain.pop()dependencies[parent].update(dependencies.pop(child))dependencies[parent].remove(child)val=subs(dsk[parent],child,val)fused.add(child)child=parentfused.add(child)ifis_renamed:rv[new_key]=valrv[child]=new_keydependencies[new_key]=dependencies[child]dependencies[child]={new_key}aliases.add(child)else:rv[child]=valforkey,valindsk.items():ifkeynotinfused:rv[key]=valifaliases:forkey,depsindependencies.items():forold_keyindeps&aliases:new_key=rv[old_key]deps.remove(old_key)deps.add(new_key)rv[key]=subs(rv[key],old_key,new_key)ifkeysisnotNone:forkeyinaliases-keys:delrv[key]deldependencies[key]returnrv,dependenciesdef_flat_set(x):ifxisNone:returnset()elifisinstance(x,set):returnxelifnotisinstance(x,(list,set)):x=[x]returnset(x)
[docs]definline(dsk,keys=None,inline_constants=True,dependencies=None):"""Return new dask with the given keys inlined with their values. Inlines all constants if ``inline_constants`` keyword is True. Note that the constant keys will remain in the graph, to remove them follow ``inline`` with ``cull``. Examples -------- >>> def inc(x): ... return x + 1 >>> def add(x, y): ... return x + y >>> d = {'x': 1, 'y': (inc, 'x'), 'z': (add, 'x', 'y')} >>> inline(d) # doctest: +ELLIPSIS {'x': 1, 'y': (<function inc at ...>, 1), 'z': (<function add at ...>, 1, 'y')} >>> inline(d, keys='y') # doctest: +ELLIPSIS {'x': 1, 'y': (<function inc at ...>, 1), 'z': (<function add at ...>, 1, (<function inc at ...>, 1))} >>> inline(d, keys='y', inline_constants=False) # doctest: +ELLIPSIS {'x': 1, 'y': (<function inc at ...>, 'x'), 'z': (<function add at ...>, 'x', (<function inc at ...>, 'x'))} """ifdependenciesandisinstance(next(iter(dependencies.values())),list):dependencies={k:set(v)fork,vindependencies.items()}keys=_flat_set(keys)ifdependenciesisNone:dependencies={k:get_dependencies(dsk,k)forkindsk}ifinline_constants:keys.update(kfork,vindsk.items()if(ishashable(v)andvindsk)or(notdependencies[k]andnotistask(v)))# Keys may depend on other keys, so determine replace order with toposort.# The values stored in `keysubs` do not include other keys.replaceorder=toposort({k:dsk[k]forkinkeysifkindsk},dependencies=dependencies)keysubs={}forkeyinreplaceorder:val=dsk[key]fordepinkeys&dependencies[key]:ifdepinkeysubs:replace=keysubs[dep]else:replace=dsk[dep]val=subs(val,dep,replace)keysubs[key]=val# Make new dask with substitutionsdsk2=keysubs.copy()forkey,valindsk.items():ifkeynotindsk2:foriteminkeys&dependencies[key]:val=subs(val,item,keysubs[item])dsk2[key]=valreturndsk2
[docs]definline_functions(dsk,output,fast_functions=None,inline_constants=False,dependencies=None):"""Inline cheap functions into larger operations Examples -------- >>> inc = lambda x: x + 1 >>> add = lambda x, y: x + y >>> double = lambda x: x * 2 >>> dsk = {'out': (add, 'i', 'd'), # doctest: +SKIP ... 'i': (inc, 'x'), ... 'd': (double, 'y'), ... 'x': 1, 'y': 1} >>> inline_functions(dsk, [], [inc]) # doctest: +SKIP {'out': (add, (inc, 'x'), 'd'), 'd': (double, 'y'), 'x': 1, 'y': 1} Protect output keys. In the example below ``i`` is not inlined because it is marked as an output key. >>> inline_functions(dsk, ['i', 'out'], [inc, double]) # doctest: +SKIP {'out': (add, 'i', (double, 'y')), 'i': (inc, 'x'), 'x': 1, 'y': 1} """ifnotfast_functions:returndskoutput=set(output)fast_functions=set(fast_functions)ifdependenciesisNone:dependencies={k:get_dependencies(dsk,k)forkindsk}dependents=reverse_dict(dependencies)definlinable(v):try:returnfunctions_of(v).issubset(fast_functions)exceptTypeError:returnFalsekeys=[kfork,vindsk.items()ifistask(v)anddependents[k]andknotinoutputandinlinable(v)]ifkeys:dsk=inline(dsk,keys,inline_constants=inline_constants,dependencies=dependencies)forkinkeys:deldsk[k]returndsk
[docs]deffunctions_of(task):"""Set of functions contained within nested task Examples -------- >>> inc = lambda x: x + 1 >>> add = lambda x, y: x + y >>> mul = lambda x, y: x * y >>> task = (add, (mul, 1, 2), (inc, 3)) # doctest: +SKIP >>> functions_of(task) # doctest: +SKIP set([add, mul, inc]) """funcs=set()work=[task]sequence_types={list,tuple}whilework:new_work=[]fortaskinwork:iftype(task)insequence_types:ifistask(task):funcs.add(unwrap_partial(task[0]))new_work.extend(task[1:])else:new_work.extend(task)work=new_workreturnfuncs
defdefault_fused_keys_renamer(keys,max_fused_key_length=120):"""Create new keys for ``fuse`` tasks. The optional parameter `max_fused_key_length` is used to limit the maximum string length for each renamed key. If this parameter is set to `None`, there is no limit. """it=reversed(keys)first_key=next(it)typ=type(first_key)ifmax_fused_key_length:# Take into account size of hash suffixmax_fused_key_length-=5def_enforce_max_key_limit(key_name):ifmax_fused_key_lengthandlen(key_name)>max_fused_key_length:name_hash=f"{hash(key_name):x}"[:4]key_name=f"{key_name[:max_fused_key_length]}-{name_hash}"returnkey_nameiftypisstr:first_name=utils.key_split(first_key)names={utils.key_split(k)forkinit}names.discard(first_name)names=sorted(names)names.append(first_key)concatenated_name="-".join(names)return_enforce_max_key_limit(concatenated_name)eliftypistupleandlen(first_key)>0andisinstance(first_key[0],str):first_name=utils.key_split(first_key)names={utils.key_split(k)forkinit}names.discard(first_name)names=sorted(names)names.append(first_key[0])concatenated_name="-".join(names)return(_enforce_max_key_limit(concatenated_name),)+first_key[1:]# PEP-484 compliant singleton constant# https://www.python.org/dev/peps/pep-0484/#support-for-singleton-types-in-unionsclassDefault(Enum):token=0def__repr__(self)->str:return"<default>"_default=Default.token
[docs]deffuse(dsk,keys=None,dependencies=None,ave_width=_default,max_width=_default,max_height=_default,max_depth_new_edges=_default,rename_keys=_default,fuse_subgraphs=_default,):"""Fuse tasks that form reductions; more advanced than ``fuse_linear`` This trades parallelism opportunities for faster scheduling by making tasks less granular. It can replace ``fuse_linear`` in optimization passes. This optimization applies to all reductions--tasks that have at most one dependent--so it may be viewed as fusing "multiple input, single output" groups of tasks into a single task. There are many parameters to fine tune the behavior, which are described below. ``ave_width`` is the natural parameter with which to compare parallelism to granularity, so it should always be specified. Reasonable values for other parameters will be determined using ``ave_width`` if necessary. Parameters ---------- dsk: dict dask graph keys: list or set, optional Keys that must remain in the returned dask graph dependencies: dict, optional {key: [list-of-keys]}. Must be a list to provide count of each key This optional input often comes from ``cull`` ave_width: float (default 1) Upper limit for ``width = num_nodes / height``, a good measure of parallelizability. dask.config key: ``optimization.fuse.ave-width`` max_width: int (default infinite) Don't fuse if total width is greater than this. dask.config key: ``optimization.fuse.max-width`` max_height: int or None (default None) Don't fuse more than this many levels. Set to None to dynamically adjust to ``1.5 + ave_width * log(ave_width + 1)``. dask.config key: ``optimization.fuse.max-height`` max_depth_new_edges: int or None (default None) Don't fuse if new dependencies are added after this many levels. Set to None to dynamically adjust to ave_width * 1.5. dask.config key: ``optimization.fuse.max-depth-new-edges`` rename_keys: bool or func, optional (default True) Whether to rename the fused keys with ``default_fused_keys_renamer`` or not. Renaming fused keys can keep the graph more understandable and comprehensive, but it comes at the cost of additional processing. If False, then the top-most key will be used. For advanced usage, a function to create the new name is also accepted. dask.config key: ``optimization.fuse.rename-keys`` fuse_subgraphs : bool or None, optional (default None) Whether to fuse multiple tasks into ``SubgraphCallable`` objects. Set to None to let the default optimizer of individual dask collections decide. If no collection-specific default exists, None defaults to False. dask.config key: ``optimization.fuse.subgraphs`` Returns ------- dsk output graph with keys fused dependencies dict mapping dependencies after fusion. Useful side effect to accelerate other downstream optimizations. """# Perform low-level fusion unless the user has# specified False explicitly.ifconfig.get("optimization.fuse.active")isFalse:returndsk,dependenciesifkeysisnotNoneandnotisinstance(keys,set):ifnotisinstance(keys,list):keys=[keys]keys=set(flatten(keys))# Read defaults from dask.yaml and/or user-defined config fileifave_widthis_default:ave_width=config.get("optimization.fuse.ave-width")assertave_widthisnot_defaultifmax_heightis_default:max_height=config.get("optimization.fuse.max-height")assertmax_heightisnot_defaultifmax_depth_new_edgesis_default:max_depth_new_edges=config.get("optimization.fuse.max-depth-new-edges")assertmax_depth_new_edgesisnot_defaultifmax_depth_new_edgesisNone:max_depth_new_edges=ave_width*1.5ifmax_widthis_default:max_width=config.get("optimization.fuse.max-width")assertmax_widthisnot_defaultifmax_widthisNone:max_width=1.5+ave_width*math.log(ave_width+1)iffuse_subgraphsis_default:fuse_subgraphs=config.get("optimization.fuse.subgraphs")assertfuse_subgraphsisnot_defaultiffuse_subgraphsisNone:fuse_subgraphs=Falseifnotave_widthornotmax_height:returndsk,dependenciesifrename_keysis_default:rename_keys=config.get("optimization.fuse.rename-keys")assertrename_keysisnot_defaultifrename_keysisTrue:key_renamer=default_fused_keys_renamerelifrename_keysisFalse:key_renamer=Noneelifnotcallable(rename_keys):raiseTypeError("rename_keys must be a boolean or callable")else:key_renamer=rename_keysrename_keys=key_renamerisnotNoneifdependenciesisNone:deps={k:get_dependencies(dsk,k,as_list=True)forkindsk}else:deps=dict(dependencies)rdeps={}fork,valsindeps.items():forvinvals:ifvnotinrdeps:rdeps[v]=[k]else:rdeps[v].append(k)deps[k]=set(vals)reducible={kfork,valsinrdeps.items()iflen(vals)==1}ifkeys:reducible-=keysfork,vindsk.items():iftype(v)isnottupleandnotisinstance(v,(numbers.Number,str)):reducible.discard(k)ifnotreducibleand(notfuse_subgraphsorall(len(set(v))!=1forvinrdeps.values())):# Quick return if there's nothing to do. Only progress if there's tasks# fusible by the main `fuse`, or by `fuse_subgraphs` if enabled.returndsk,depsrv=dsk.copy()fused_trees={}# These are the stacks we use to store data as we traverse the graphinfo_stack=[]children_stack=[]# For speeddeps_pop=deps.popreducible_add=reducible.addreducible_pop=reducible.popreducible_remove=reducible.removefused_trees_pop=fused_trees.popinfo_stack_append=info_stack.appendinfo_stack_pop=info_stack.popchildren_stack_append=children_stack.appendchildren_stack_extend=children_stack.extendchildren_stack_pop=children_stack.popwhilereducible:parent=reducible_pop()reducible_add(parent)whileparentinreducible:# Go to the topparent=rdeps[parent][0]children_stack_append(parent)children_stack_extend(reducible&deps[parent])whileTrue:child=children_stack[-1]ifchild!=parent:children=reducible&deps[child]whilechildren:# Depth-first searchchildren_stack_extend(children)parent=childchild=children_stack[-1]children=reducible&deps[child]children_stack_pop()# This is a leaf node in the reduction region# key, task, fused_keys, height, width, number of nodes, fudge, set of edgesinfo_stack_append((child,rv[child],[child]ifrename_keyselseNone,1,1,1,0,deps[child]-reducible,))else:children_stack_pop()# Calculate metrics and fuse as appropriatedeps_parent=deps[parent]edges=deps_parent-reduciblechildren=deps_parent-edgesnum_children=len(children)ifnum_children==1:(child_key,child_task,child_keys,height,width,num_nodes,fudge,children_edges,)=info_stack_pop()num_children_edges=len(children_edges)iffudge>num_children_edges-1>=0:fudge=num_children_edges-1edges|=children_edgesno_new_edges=len(edges)==num_children_edgesifnotno_new_edges:fudge+=1if((num_nodes+fudge)/height<=ave_widthand# Sanity check; don't go too deep if new levels introduce new edge dependencies(no_new_edgesorheight<max_depth_new_edges)):# Perform substitutions as we goval=subs(dsk[parent],child_key,child_task)deps_parent.remove(child_key)deps_parent|=deps_pop(child_key)delrv[child_key]reducible_remove(child_key)ifrename_keys:child_keys.append(parent)fused_trees[parent]=child_keysfused_trees_pop(child_key,None)ifchildren_stack:ifno_new_edges:# Linear fuseinfo_stack_append((parent,val,child_keys,height,width,num_nodes,fudge,edges,))else:info_stack_append((parent,val,child_keys,height+1,width,num_nodes+1,fudge,edges,))else:rv[parent]=valbreakelse:rv[child_key]=child_taskreducible_remove(child_key)ifchildren_stack:# Allow the parent to be fused, but only under strict circumstances.# Ensure that linear chains may still be fused.iffudge>int(ave_width-1):fudge=int(ave_width-1)# This task *implicitly* depends on `edges`info_stack_append((parent,rv[parent],[parent]ifrename_keyselseNone,1,width,1,fudge,edges,))else:breakelse:child_keys=[]height=1width=0num_single_nodes=0num_nodes=0fudge=0children_edges=set()max_num_edges=0children_info=info_stack[-num_children:]delinfo_stack[-num_children:]for(_,_,_,cur_height,cur_width,cur_num_nodes,cur_fudge,cur_edges,)inchildren_info:ifcur_height==1:num_single_nodes+=1elifcur_height>height:height=cur_heightwidth+=cur_widthnum_nodes+=cur_num_nodesfudge+=cur_fudgeiflen(cur_edges)>max_num_edges:max_num_edges=len(cur_edges)children_edges|=cur_edges# Fudge factor to account for possible parallelism with the boundariesnum_children_edges=len(children_edges)fudge+=min(num_children-1,max(0,num_children_edges-max_num_edges))iffudge>num_children_edges-1>=0:fudge=num_children_edges-1edges|=children_edgesno_new_edges=len(edges)==num_children_edgesifnotno_new_edges:fudge+=1if((num_nodes+fudge)/height<=ave_widthandnum_single_nodes<=ave_widthandwidth<=max_widthandheight<=max_heightand# Sanity check; don't go too deep if new levels introduce new edge dependencies(no_new_edgesorheight<max_depth_new_edges)):# Perform substitutions as we goval=dsk[parent]children_deps=set()forchild_infoinchildren_info:cur_child=child_info[0]val=subs(val,cur_child,child_info[1])delrv[cur_child]children_deps|=deps_pop(cur_child)reducible_remove(cur_child)ifrename_keys:fused_trees_pop(cur_child,None)child_keys.extend(child_info[2])deps_parent-=childrendeps_parent|=children_depsifrename_keys:child_keys.append(parent)fused_trees[parent]=child_keysifchildren_stack:info_stack_append((parent,val,child_keys,height+1,width,num_nodes+1,fudge,edges,))else:rv[parent]=valbreakelse:forchild_infoinchildren_info:rv[child_info[0]]=child_info[1]reducible_remove(child_info[0])ifchildren_stack:# Allow the parent to be fused, but only under strict circumstances.# Ensure that linear chains may still be fused.ifwidth>max_width:width=max_widthiffudge>int(ave_width-1):fudge=int(ave_width-1)# key, task, height, width, number of nodes, fudge, set of edges# This task *implicitly* depends on `edges`info_stack_append((parent,rv[parent],[parent]ifrename_keyselseNone,1,width,1,fudge,edges,))else:break# Traverse upwardsparent=rdeps[parent][0]iffuse_subgraphs:_inplace_fuse_subgraphs(rv,keys,deps,fused_trees,rename_keys)ifkey_renamer:forroot_key,fused_keysinfused_trees.items():alias=key_renamer(fused_keys)ifaliasisnotNoneandaliasnotinrv:rv[alias]=rv[root_key]rv[root_key]=aliasdeps[alias]=deps[root_key]deps[root_key]={alias}returnrv,deps
def_inplace_fuse_subgraphs(dsk,keys,dependencies,fused_trees,rename_keys):"""Subroutine of fuse. Mutates dsk, dependencies, and fused_trees inplace"""# locate all members of linear chainschild2parent={}unfusible=set()forparentindsk:deps=dependencies[parent]has_many_children=len(deps)>1forchildindeps:ifkeysisnotNoneandchildinkeys:unfusible.add(child)elifchildinchild2parent:delchild2parent[child]unfusible.add(child)elifhas_many_children:unfusible.add(child)elifchildnotinunfusible:child2parent[child]=parent# construct the chains from ancestor to descendantchains=[]parent2child={v:kfork,vinchild2parent.items()}whilechild2parent:child,parent=child2parent.popitem()chain=[child,parent]whileparentinchild2parent:parent=child2parent.pop(parent)delparent2child[parent]chain.append(parent)chain.reverse()whilechildinparent2child:child=parent2child.pop(child)delchild2parent[child]chain.append(child)# Skip chains with < 2 executable tasksntasks=0forkeyinchain:ntasks+=istask(dsk[key])ifntasks>1:chains.append(chain)break# Mutate dsk fusing chains into subgraphsforchaininchains:subgraph={k:dsk[k]forkinchain}outkey=chain[0]# Update dependencies and graphinkeys_set=dependencies[outkey]=dependencies[chain[-1]]forkinchain[1:]:deldependencies[k]deldsk[k]# Create new taskinkeys=tuple(inkeys_set)dsk[outkey]=(SubgraphCallable(subgraph,outkey,inkeys),)+inkeys# Mutate `fused_trees` if key renaming is needed (renaming done in fuse)ifrename_keys:chain2=[]forkinchain:subchain=fused_trees.pop(k,False)ifsubchain:chain2.extend(subchain)else:chain2.append(k)fused_trees[outkey]=chain2classSubgraphCallable:"""Create a callable object from a dask graph. Parameters ---------- dsk : dict A dask graph outkey : Dask key The output key from the graph inkeys : list A list of keys to be used as arguments to the callable. name : str, optional The name to use for the function. """__slots__=("dsk","outkey","inkeys","name")def__init__(self,dsk,outkey,inkeys,name=None):self.dsk=dskself.outkey=outkeyself.inkeys=inkeysifnameisNone:name=f"subgraph_callable-{uuid.uuid4()}"self.name=namedef__repr__(self):returnself.namedef__eq__(self,other):return(type(self)istype(other)andself.name==other.nameandself.outkey==other.outkeyandset(self.inkeys)==set(other.inkeys))def__ne__(self,other):returnnot(self==other)def__call__(self,*args):ifnotlen(args)==len(self.inkeys):raiseValueError("Expected %d args, got %d"%(len(self.inkeys),len(args)))returncore.get(self.dsk,self.outkey,dict(zip(self.inkeys,args)))def__reduce__(self):return(SubgraphCallable,(self.dsk,self.outkey,self.inkeys,self.name))def__hash__(self):returnhash(tuple((self.outkey,frozenset(self.inkeys),self.name)))