Browse Source

HPCC-13275 Split the child dependency spotting from the resourcer

This change is part of the preparation for improving the execution of conditional
expressions.  By splitting the child dataset spotting from the resourcing it
allows other transformations to also be applied within graph "resourcing".

The change looks substantial, but some of that is moving code and refactoring it.
The main changes are:
- Split the child dataset spotting into a separate stage
- Move some logic into the the options class which is shared between stages
- Move the dependency tracking code into a separate class, and reduce the lifetime
  of some of the variables.
- Allow graph results to be reused by the resourcer
- Introduce the concept of a _graphLocal_ action which is only executed when required
  by another activity.

Signed-off-by: Gavin Halliday <gavin.halliday@lexisnexis.com>
Gavin Halliday 10 years ago
parent
commit
fa5bdebbbd

+ 4 - 0
ecl/hql/hqlatoms.cpp

@@ -189,6 +189,7 @@ IAtom * gctxmethodAtom;
 IAtom * getAtom;
 IAtom * getAtom;
 IAtom * globalAtom;
 IAtom * globalAtom;
 IAtom * graphAtom;
 IAtom * graphAtom;
+IAtom * _graphLocal_Atom;
 IAtom * groupAtom;
 IAtom * groupAtom;
 IAtom * groupedAtom;
 IAtom * groupedAtom;
 IAtom * hashAtom;
 IAtom * hashAtom;
@@ -226,6 +227,7 @@ IAtom * keyedAtom;
 IAtom * labeledAtom;
 IAtom * labeledAtom;
 IAtom * languageAtom;
 IAtom * languageAtom;
 IAtom * lastAtom;
 IAtom * lastAtom;
+IAtom * _lazy_Atom;
 IAtom * leftAtom;
 IAtom * leftAtom;
 IAtom * leftonlyAtom;
 IAtom * leftonlyAtom;
 IAtom * leftouterAtom;
 IAtom * leftouterAtom;
@@ -622,6 +624,7 @@ MODULE_INIT(INIT_PRIORITY_HQLATOM)
     MAKEATOM(global);
     MAKEATOM(global);
     MAKEATOM(globalContext);
     MAKEATOM(globalContext);
     MAKEATOM(graph);
     MAKEATOM(graph);
+    MAKESYSATOM(graphLocal);
     MAKEATOM(group);
     MAKEATOM(group);
     MAKEATOM(grouped);
     MAKEATOM(grouped);
     MAKEATOM(hash);
     MAKEATOM(hash);
@@ -660,6 +663,7 @@ MODULE_INIT(INIT_PRIORITY_HQLATOM)
     MAKEATOM(labeled);
     MAKEATOM(labeled);
     MAKEATOM(language);
     MAKEATOM(language);
     MAKEATOM(last);
     MAKEATOM(last);
+    MAKESYSATOM(lazy);
     MAKEATOM(left);
     MAKEATOM(left);
     leftonlyAtom = createLowerCaseAtom("left only");
     leftonlyAtom = createLowerCaseAtom("left only");
     leftouterAtom = createLowerCaseAtom("left outer");
     leftouterAtom = createLowerCaseAtom("left outer");

+ 1 - 0
ecl/hql/hqlatoms.hpp

@@ -229,6 +229,7 @@ extern HQL_API IAtom * keyedAtom;
 extern HQL_API IAtom * labeledAtom;
 extern HQL_API IAtom * labeledAtom;
 extern HQL_API IAtom * languageAtom;
 extern HQL_API IAtom * languageAtom;
 extern HQL_API IAtom * lastAtom;
 extern HQL_API IAtom * lastAtom;
+extern HQL_API IAtom * _lazy_Atom;
 extern HQL_API IAtom * leftAtom;
 extern HQL_API IAtom * leftAtom;
 extern HQL_API IAtom * leftonlyAtom;
 extern HQL_API IAtom * leftonlyAtom;
 extern HQL_API IAtom * leftouterAtom;
 extern HQL_API IAtom * leftouterAtom;

+ 10 - 0
ecl/hql/hqlutil.cpp

@@ -619,6 +619,16 @@ static IHqlExpression * findCommonExpression(IHqlExpression * lower, IHqlExpress
 
 
 //---------------------------------------------------------------------------------------------------------------------
 //---------------------------------------------------------------------------------------------------------------------
 
 
+bool isFileOutput(IHqlExpression * expr)
+{
+    return (expr->getOperator() == no_output) && (queryRealChild(expr, 1) != NULL);
+}
+
+bool isWorkunitOutput(IHqlExpression * expr)
+{
+    return (expr->getOperator() == no_output) && (queryRealChild(expr, 1) == NULL);
+}
+
 bool isCommonSubstringRange(IHqlExpression * expr)
 bool isCommonSubstringRange(IHqlExpression * expr)
 {
 {
     if (expr->getOperator() != no_substring)
     if (expr->getOperator() != no_substring)

+ 2 - 0
ecl/hql/hqlutil.hpp

@@ -681,6 +681,8 @@ extern HQL_API IHqlExpression * queryTransformAssign(IHqlExpression * transform,
 extern HQL_API IHqlExpression * queryTransformAssignValue(IHqlExpression * transform, IHqlExpression * searchField);
 extern HQL_API IHqlExpression * queryTransformAssignValue(IHqlExpression * transform, IHqlExpression * searchField);
 
 
 extern HQL_API bool isCommonSubstringRange(IHqlExpression * expr);
 extern HQL_API bool isCommonSubstringRange(IHqlExpression * expr);
+extern HQL_API bool isFileOutput(IHqlExpression * expr);
+extern HQL_API bool isWorkunitOutput(IHqlExpression * expr);
 
 
 class HQL_API AtmostLimit
 class HQL_API AtmostLimit
 {
 {

+ 0 - 2
ecl/hqlcpp/hqlcatom.cpp

@@ -93,7 +93,6 @@ IAtom * _spill_Atom;
 IAtom * _spillReason_Atom;
 IAtom * _spillReason_Atom;
 IAtom * _steppedMeta_Atom;
 IAtom * _steppedMeta_Atom;
 IAtom * subgraphAtom;
 IAtom * subgraphAtom;
-IAtom * _tempCount_Atom;
 IAtom * _translated_Atom;
 IAtom * _translated_Atom;
 IAtom * utf8Atom;
 IAtom * utf8Atom;
 IAtom * wrapperAtom;
 IAtom * wrapperAtom;
@@ -1488,7 +1487,6 @@ MODULE_INIT(INIT_PRIORITY_HQLATOM-1)
     MAKESYSATOM(spill);
     MAKESYSATOM(spill);
     MAKESYSATOM(spillReason);
     MAKESYSATOM(spillReason);
     MAKESYSATOM(steppedMeta);
     MAKESYSATOM(steppedMeta);
-    MAKESYSATOM(tempCount);
     MAKESYSATOM(translated);
     MAKESYSATOM(translated);
     return true;
     return true;
 }
 }

+ 0 - 1
ecl/hqlcpp/hqlcatom.hpp

@@ -93,7 +93,6 @@ extern IAtom * _spill_Atom;
 extern IAtom * _spillReason_Atom;
 extern IAtom * _spillReason_Atom;
 extern IAtom * _steppedMeta_Atom;
 extern IAtom * _steppedMeta_Atom;
 extern IAtom * subgraphAtom;
 extern IAtom * subgraphAtom;
-extern IAtom * _tempCount_Atom;
 extern IAtom * _translated_Atom;
 extern IAtom * _translated_Atom;
 extern IAtom * utf8Atom;
 extern IAtom * utf8Atom;
 extern IAtom * wrapperAtom;
 extern IAtom * wrapperAtom;

+ 1 - 0
ecl/hqlcpp/hqlcpp.cpp

@@ -1759,6 +1759,7 @@ void HqlCppTranslator::cacheOptions()
         DebugOption(options.expandSelectCreateRow,"expandSelectCreateRow",false),
         DebugOption(options.expandSelectCreateRow,"expandSelectCreateRow",false),
         DebugOption(options.optimizeSortAllFields,"optimizeSortAllFields",true),
         DebugOption(options.optimizeSortAllFields,"optimizeSortAllFields",true),
         DebugOption(options.optimizeSortAllFieldsStrict,"optimizeSortAllFieldsStrict",false),
         DebugOption(options.optimizeSortAllFieldsStrict,"optimizeSortAllFieldsStrict",false),
+        DebugOption(options.alwaysReuseGlobalSpills,"alwaysReuseGlobalSpills",true),
     };
     };
 
 
     //get options values from workunit
     //get options values from workunit

+ 1 - 0
ecl/hqlcpp/hqlcpp.ipp

@@ -746,6 +746,7 @@ struct HqlCppOptions
     bool                expandSelectCreateRow;
     bool                expandSelectCreateRow;
     bool                optimizeSortAllFields;
     bool                optimizeSortAllFields;
     bool                optimizeSortAllFieldsStrict;
     bool                optimizeSortAllFieldsStrict;
+    bool                alwaysReuseGlobalSpills;
 };
 };
 
 
 //Any information gathered while processing the query should be moved into here, rather than cluttering up the translator class
 //Any information gathered while processing the query should be moved into here, rather than cluttering up the translator class

+ 1 - 1
ecl/hqlcpp/hqlhtcpp.cpp

@@ -7688,7 +7688,7 @@ static bool isFilePersist(IHqlExpression * expr)
             expr = expr->queryChild(1);
             expr = expr->queryChild(1);
             break;
             break;
         case no_output:
         case no_output:
-            return (queryRealChild(expr, 1) != NULL);
+            return isFileOutput(expr);
         case no_actionlist:
         case no_actionlist:
         case no_orderedactionlist:
         case no_orderedactionlist:
             expr = expr->queryChild(expr->numChildren()-1);
             expr = expr->queryChild(expr->numChildren()-1);

File diff suppressed because it is too large
+ 3281 - 2565
ecl/hqlcpp/hqlresource.cpp


+ 65 - 94
ecl/hqlcpp/hqlresource.ipp

@@ -38,44 +38,21 @@ enum ResourceType {
 class CResourceOptions
 class CResourceOptions
 {
 {
 public:
 public:
-    CResourceOptions(UniqueSequenceCounter & _spillSequence)
-    : spillSequence(_spillSequence)
-    {
-        filteredSpillThreshold = 0;
-        minimizeSpillSize = 0;
-        allowThroughSpill = false;
-        allowThroughResult = false;
-        cloneFilteredIndex = false;
-        spillSharedConditionals = false;
-        shareDontExpand = false;
-        useGraphResults = false;
-        noConditionalLinks = false;
-        minimiseSpills = false;
-        hoistResourced = false;
-        isChildQuery = false;
-        groupedChildIterators = false;
-        allowSplitBetweenSubGraphs = false;
-        preventKeyedSplit = false;
-        preventSteppedSplit = false;
-        minimizeSkewBeforeSpill = false;
-        expandSingleConstRow = false;
-        createSpillAsDataset = false;
-        optimizeSharedInputs = false;
-        combineSiblings = false;
-        actionLinkInNewGraph = false;
-        convertCompoundToExecuteWhen = false;
-        useResultsForChildSpills = false;
-        alwaysUseGraphResults = false;
-        newBalancedSpotter = false;
-        graphIdExpr = NULL;
-        nextResult = 0;
-        clusterSize = 0;
-        targetClusterType = ThorLCRCluster;
-        state.updateSequence = 0;
-    }
+    CResourceOptions(ClusterType _targetClusterType, unsigned _clusterSize, const HqlCppOptions & _translatorOptions, UniqueSequenceCounter & _spillSequence);
+
+    IHqlExpression * createResultSpillName();
+    IHqlExpression * createDiskSpillName();
+    IHqlExpression * createGlobalSpillName();
 
 
-    IHqlExpression * createSpillName(bool isGraphResult);
     void noteGraphsChanged() { state.updateSequence++; }
     void noteGraphsChanged() { state.updateSequence++; }
+    void setChildQuery(bool value);
+    void setNewChildQuery(IHqlExpression * graphIdExpr, unsigned numResults);
+    void setUseGraphResults(bool _useGraphResults)
+    {
+        useGraphResults = _useGraphResults;
+    }
+    bool useGraphResult(bool linkedFromChild);
+    bool useGlobalResult(bool linkedFromChild);
 
 
 public:
 public:
     UniqueSequenceCounter & spillSequence;
     UniqueSequenceCounter & spillSequence;
@@ -105,6 +82,9 @@ public:
     bool     useResultsForChildSpills;
     bool     useResultsForChildSpills;
     bool     alwaysUseGraphResults;
     bool     alwaysUseGraphResults;
     bool     newBalancedSpotter;
     bool     newBalancedSpotter;
+    bool     spillMultiCondition;
+    bool     spotThroughAggregate;
+    bool     alwaysReuseGlobalSpills;
 
 
     IHqlExpression * graphIdExpr;
     IHqlExpression * graphIdExpr;
     unsigned nextResult;
     unsigned nextResult;
@@ -160,7 +140,8 @@ public:
     virtual void changeSourceGraph(ResourceGraphInfo * newGraph);
     virtual void changeSourceGraph(ResourceGraphInfo * newGraph);
     virtual void changeSinkGraph(ResourceGraphInfo * newGraph);
     virtual void changeSinkGraph(ResourceGraphInfo * newGraph);
     virtual bool isRedundantLink();
     virtual bool isRedundantLink();
-    virtual bool isDependency() { return false; }
+    virtual bool isDependency() const { return false; }
+    virtual IHqlExpression * queryDependency() const { return NULL; }
 
 
 protected:
 protected:
     void trace(const char * name);
     void trace(const char * name);
@@ -176,12 +157,16 @@ public:
 class ResourceGraphDependencyLink : public ResourceGraphLink
 class ResourceGraphDependencyLink : public ResourceGraphLink
 {
 {
 public:
 public:
-    ResourceGraphDependencyLink(ResourceGraphInfo * _sourceGraph, IHqlExpression * _sourceNode, ResourceGraphInfo * _sinkGraph, IHqlExpression * _sinkNode);
+    ResourceGraphDependencyLink(ResourceGraphInfo * _sourceGraph, IHqlExpression * _sourceNode, ResourceGraphInfo * _sinkGraph, IHqlExpression * _sinkNode, IHqlExpression * _dependency);
 
 
     virtual void changeSourceGraph(ResourceGraphInfo * newGraph);
     virtual void changeSourceGraph(ResourceGraphInfo * newGraph);
     virtual void changeSinkGraph(ResourceGraphInfo * newGraph);
     virtual void changeSinkGraph(ResourceGraphInfo * newGraph);
     virtual bool isRedundantLink()          { return false; }
     virtual bool isRedundantLink()          { return false; }
-    virtual bool isDependency() { return true; }
+    virtual bool isDependency() const { return true; }
+    virtual IHqlExpression * queryDependency() const { return dependency; }
+
+protected:
+    LinkedHqlExpr dependency;
 };
 };
 
 
 typedef CIArrayOf<ResourceGraphInfo> ResourceGraphArray;
 typedef CIArrayOf<ResourceGraphInfo> ResourceGraphArray;
@@ -204,7 +189,7 @@ public:
     bool isDependentOn(ResourceGraphInfo & other, bool allowDirect);
     bool isDependentOn(ResourceGraphInfo & other, bool allowDirect);
     bool isVeryCheap();
     bool isVeryCheap();
     bool mergeInSibling(ResourceGraphInfo & other, const CResources & limit);
     bool mergeInSibling(ResourceGraphInfo & other, const CResources & limit);
-    bool mergeInSource(ResourceGraphInfo & other, const CResources & limit);
+    bool mergeInSource(ResourceGraphInfo & other, const CResources & limit, bool ignoreConditional);
     void removeResources(const CResources & value);
     void removeResources(const CResources & value);
 
 
     bool isSharedInput(IHqlExpression * expr);
     bool isSharedInput(IHqlExpression * expr);
@@ -327,14 +312,40 @@ public:
     bool ignoreExternalDependencies;
     bool ignoreExternalDependencies;
 };
 };
 
 
-class ResourcerInfo : public CInterfaceOf<IInterface>
+class SpillerInfo : public NewTransformInfo
+{
+public:
+    SpillerInfo(IHqlExpression * _original, CResourceOptions * _options);
+
+    IHqlExpression * createSpilledRead(IHqlExpression * spillReason);
+    IHqlExpression * createSpilledWrite(IHqlExpression * transformed, bool lazy);
+    bool isUsedFromChild() const { return linkedFromChild; }
+    IHqlExpression * queryOutputSpillFile() const { return outputToUseForSpill; }
+    void setPotentialSpillFile(IHqlExpression * expr);
+
+
+protected:
+    void addSpillFlags(HqlExprArray & args, bool isRead);
+    IHqlExpression * createSpillName();
+    bool useGraphResult();
+    bool useGlobalResult();
+    IHqlExpression * wrapRowOwn(IHqlExpression * expr);
+
+protected:
+    CResourceOptions * options;
+    HqlExprAttr spillName;
+    HqlExprAttr spilledDataset;
+    IHqlExpression * outputToUseForSpill;
+    bool linkedFromChild; // could reuse a spare byte in the parent class
+};
+
+class ResourcerInfo : public SpillerInfo
 {
 {
 public:
 public:
     enum { PathUnknown, PathConditional, PathUnconditional };
     enum { PathUnknown, PathConditional, PathUnconditional };
 
 
     ResourcerInfo(IHqlExpression * _original, CResourceOptions * _options);
     ResourcerInfo(IHqlExpression * _original, CResourceOptions * _options);
 
 
-    IHqlExpression * createSpilledRead(IHqlExpression * spillReason);
     IHqlExpression * createTransformedExpr(IHqlExpression * expr);
     IHqlExpression * createTransformedExpr(IHqlExpression * expr);
 
 
     bool addCondition(IHqlExpression * condition);
     bool addCondition(IHqlExpression * condition);
@@ -356,8 +367,12 @@ public:
     void resetBalanced();
     void resetBalanced();
     void setConditionSource(IHqlExpression * condition, bool isFirst);
     void setConditionSource(IHqlExpression * condition, bool isFirst);
 
 
-    //hthor - don't merge anything to a global result because we don't allow splitters.
-    inline bool preventMerge()          { return !options->canSplit() && useGlobalResult(); }
+    inline bool preventMerge()
+    {
+        //If we need to create a spill global result, but engine can't split then don't merge
+        //Only required because hthor doesn't support splitters (or through-workunit results).
+        return !options->canSplit() && options->useGlobalResult(linkedFromChild);
+    }
     inline bool isUnconditional()       { return (pathToExpr == ResourcerInfo::PathUnconditional); }
     inline bool isUnconditional()       { return (pathToExpr == ResourcerInfo::PathUnconditional); }
     inline bool isConditionExpr()
     inline bool isConditionExpr()
     {
     {
@@ -393,34 +408,20 @@ public:
 
 
 protected:
 protected:
     bool spillSharesSplitter();
     bool spillSharesSplitter();
-    bool useGraphResult();
-    bool useGlobalResult();
     IHqlExpression * createAggregation(IHqlExpression * expr);
     IHqlExpression * createAggregation(IHqlExpression * expr);
-    IHqlExpression * createSpilledWrite(IHqlExpression * transformed);
     IHqlExpression * createSpiller(IHqlExpression * transformed, bool reuseSplitter);
     IHqlExpression * createSpiller(IHqlExpression * transformed, bool reuseSplitter);
     IHqlExpression * createSplitter(IHqlExpression * transformed);
     IHqlExpression * createSplitter(IHqlExpression * transformed);
 
 
-protected:
-    void addSpillFlags(HqlExprArray & args, bool isRead);
-    IHqlExpression * createSpillName();
-    IHqlExpression * wrapRowOwn(IHqlExpression * expr);
-
 public:
 public:
-    HqlExprAttr original;
     Owned<ResourceGraphInfo> graph;
     Owned<ResourceGraphInfo> graph;
-    HqlExprAttr spillName;
-    IHqlExpression * transformed;
-    IHqlExpression * outputToUseForSpill;
-    CResourceOptions * options;
     HqlExprAttr pathToSplitter;
     HqlExprAttr pathToSplitter;
     HqlExprArray aggregates;
     HqlExprArray aggregates;
     HqlExprArray conditions;
     HqlExprArray conditions;
-    ChildDependentArray childDependents;
-    HqlExprAttr spilledDataset;
     HqlExprAttr splitterOutput;
     HqlExprAttr splitterOutput;
     HqlExprArray projected;
     HqlExprArray projected;
     HqlExprAttr projectedExpr;
     HqlExprAttr projectedExpr;
     CIArrayOf<CSplitterLink> balancedLinks;
     CIArrayOf<CSplitterLink> balancedLinks;
+    GraphLinkArray dependsOn;           // NB: These do no link....
 
 
     unsigned numUses;
     unsigned numUses;
     unsigned numExternalUses;
     unsigned numExternalUses;
@@ -440,7 +441,6 @@ public:
     bool isSpillPoint:1;
     bool isSpillPoint:1;
     bool balanced:1;
     bool balanced:1;
     bool isAlreadyInScope:1;
     bool isAlreadyInScope:1;
-    bool linkedFromChild:1;
     bool forceHoist:1;
     bool forceHoist:1;
     bool neverSplit:1;
     bool neverSplit:1;
     bool isConditionalFilter:1;
     bool isConditionalFilter:1;
@@ -449,32 +449,19 @@ public:
     bool balancedVisiting:1;
     bool balancedVisiting:1;
 };
 };
 
 
-struct DependencySourceInfo
-{
-    HqlExprArray                    search;
-    CIArrayOf<ResourceGraphInfo>    graphs;
-    HqlExprArray                    exprs;
-};
-
-
+class EclResourceDependencyGatherer;
 class EclResourcer
 class EclResourcer
 {
 {
     friend class SelectHoistTransformer;
     friend class SelectHoistTransformer;
     friend class CSplitterInfo;
     friend class CSplitterInfo;
 public:
 public:
-    EclResourcer(IErrorReceiver & _errors, IConstWorkUnit * _wu, ClusterType _targetClusterType, unsigned _clusterSize, const HqlCppOptions & _translatorOptions, UniqueSequenceCounter & _spillSequence);
+    EclResourcer(IErrorReceiver & _errors, IConstWorkUnit * _wu, const HqlCppOptions & _translatorOptions, CResourceOptions & _options);
     ~EclResourcer();
     ~EclResourcer();
 
 
     void resourceGraph(IHqlExpression * expr, HqlExprArray & transformed);
     void resourceGraph(IHqlExpression * expr, HqlExprArray & transformed);
     void resourceRemoteGraph(IHqlExpression * expr, HqlExprArray & transformed);
     void resourceRemoteGraph(IHqlExpression * expr, HqlExprArray & transformed);
-    void setChildQuery(bool value);
-    void setNewChildQuery(IHqlExpression * graphIdExpr, unsigned numResults);
     void setSequential(bool _sequential) { sequential = _sequential; }
     void setSequential(bool _sequential) { sequential = _sequential; }
-    void setUseGraphResults(bool _useGraphResults) 
-    { 
-        options.useGraphResults = _useGraphResults; 
-    }
-    void tagActiveCursors(HqlExprCopyArray & activeRows);
+    void tagActiveCursors(HqlExprCopyArray * activeRows);
     inline unsigned numGraphResults() { return options.nextResult; }
     inline unsigned numGraphResults() { return options.nextResult; }
 
 
 protected:
 protected:
@@ -487,15 +474,11 @@ protected:
     void replaceGraphReferences(ResourceGraphInfo * oldGraph, ResourceGraphInfo * newGraph);
     void replaceGraphReferences(ResourceGraphInfo * oldGraph, ResourceGraphInfo * newGraph);
 
 
 //Pass 1
 //Pass 1
-    void gatherChildSplitPoints(IHqlExpression * expr, ResourcerInfo * info, unsigned first, unsigned last);
     bool findSplitPoints(IHqlExpression * expr, bool isProjected);
     bool findSplitPoints(IHqlExpression * expr, bool isProjected);
     void findSplitPoints(HqlExprArray & exprs);
     void findSplitPoints(HqlExprArray & exprs);
     void noteConditionalChildren(BoolArray & alwaysHoistChild);
     void noteConditionalChildren(BoolArray & alwaysHoistChild);
     void deriveUsageCounts(IHqlExpression * expr);
     void deriveUsageCounts(IHqlExpression * expr);
     void deriveUsageCounts(const HqlExprArray & exprs);
     void deriveUsageCounts(const HqlExprArray & exprs);
-    void extendSplitPoints();
-    void projectChildDependents();
-    IHqlExpression * projectChildDependent(IHqlExpression * expr);
 
 
 //Pass 2
 //Pass 2
     void createInitialGraph(IHqlExpression * expr, IHqlExpression * owner, ResourceGraphInfo * ownerGraph, LinkKind linkKind, bool forceNewGraph);
     void createInitialGraph(IHqlExpression * expr, IHqlExpression * owner, ResourceGraphInfo * ownerGraph, LinkKind linkKind, bool forceNewGraph);
@@ -520,13 +503,7 @@ protected:
     void resourceSubGraphs(HqlExprArray & exprs);
     void resourceSubGraphs(HqlExprArray & exprs);
 
 
 //Pass 5
 //Pass 5
-    void addDependencySource(IHqlExpression * search, ResourceGraphInfo * curGraph, IHqlExpression * expr);
-    void addDependencyUse(IHqlExpression * search, ResourceGraphInfo * curGraph, IHqlExpression * expr);
-    bool addExprDependency(IHqlExpression * expr, ResourceGraphInfo * curGraph, IHqlExpression * activityExpr);
-    void addRefExprDependency(IHqlExpression * expr, ResourceGraphInfo * curGraph, IHqlExpression * activityExpr);
-    void doAddChildDependencies(IHqlExpression * expr, ResourceGraphInfo * graph, IHqlExpression * activityExpr);
-    void addChildDependencies(IHqlExpression * expr, ResourceGraphInfo * graph, IHqlExpression * activityExpr);
-    void addDependencies(IHqlExpression * expr, ResourceGraphInfo * graph, IHqlExpression * activityExpr);
+    void addDependencies(EclResourceDependencyGatherer & gatherer, IHqlExpression * expr, ResourceGraphInfo * graph, IHqlExpression * activityExpr);
     void addDependencies(HqlExprArray & exprs);
     void addDependencies(HqlExprArray & exprs);
 
 
 //Pass 6
 //Pass 6
@@ -560,7 +537,6 @@ protected:
     void moveExternalSpillPoints();
     void moveExternalSpillPoints();
 
 
 //Pass 9
 //Pass 9
-    IHqlExpression * replaceResourcedReferences(ResourcerInfo * info, IHqlExpression * expr);
     IHqlExpression * doCreateResourced(IHqlExpression * expr, ResourceGraphInfo * graph, bool expandInParent, bool defineSideEffect);
     IHqlExpression * doCreateResourced(IHqlExpression * expr, ResourceGraphInfo * graph, bool expandInParent, bool defineSideEffect);
     IHqlExpression * createResourced(IHqlExpression * expr, ResourceGraphInfo * graph, bool expandInParent, bool defineSideEffect);
     IHqlExpression * createResourced(IHqlExpression * expr, ResourceGraphInfo * graph, bool expandInParent, bool defineSideEffect);
     void createResourced(HqlExprArray & transformed);
     void createResourced(HqlExprArray & transformed);
@@ -582,21 +558,16 @@ protected:
     CIArrayOf<ResourceGraphInfo> graphs;
     CIArrayOf<ResourceGraphInfo> graphs;
     CIArrayOf<ResourceGraphLink> links;
     CIArrayOf<ResourceGraphLink> links;
     ClusterType targetClusterType;
     ClusterType targetClusterType;
-    DependencySourceInfo dependencySource;                  
-    unsigned clusterSize;
     CResources * resourceLimit;
     CResources * resourceLimit;
     IErrorReceiver * errors;
     IErrorReceiver * errors;
     unsigned thisPass;
     unsigned thisPass;
     bool spilled;
     bool spilled;
-    bool spillMultiCondition;
-    bool spotThroughAggregate;
     bool insideNeverSplit;
     bool insideNeverSplit;
     bool insideSteppedNeverSplit;
     bool insideSteppedNeverSplit;
     bool sequential;
     bool sequential;
-    CResourceOptions options;
+    CResourceOptions & options;
     HqlExprArray rootConditions;
     HqlExprArray rootConditions;
     HqlExprCopyArray activeSelectors;
     HqlExprCopyArray activeSelectors;
-    ChildDependentArray allChildDependents;
 };
 };
 
 
 #endif
 #endif

+ 58 - 0
testing/regress/ecl/childds1.ecl

@@ -0,0 +1,58 @@
+/*##############################################################################
+
+    HPCC SYSTEMS software Copyright (C) 2015 HPCC Systems.
+
+    Licensed 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.
+############################################################################## */
+
+//This is needed to prevent cntBad being hoisted before the resourcing, and becoming unconditional
+//The tests are part of the work aiming to remove this code.
+#option ('workunitTemporaries', false);
+
+idRec := { unsigned id; };
+mainRec := { unsigned seq, dataset(idRec) ids };
+
+idRec createId(unsigned id) := TRANSFORM
+    SELF.id := id;
+END; 
+
+mainRec createMain(unsigned c, unsigned num) := TRANSFORM
+    SELF.seq := c;
+    SELF.ids := DATASET(num, createId(c + (COUNTER-1)));
+END;
+
+ds := NOFOLD(DATASET(4, createMain(COUNTER, 3)));
+
+boolean assertTrue(boolean x, const varstring msg = 'Condition should have been true') := BEGINC++
+    #option pure
+    if (!x)
+        rtlFail(0, msg);
+    return x;
+ENDC++;
+
+trueValue := true : stored('trueValue');
+falseValue := false : stored('falseValue');
+
+//Case 1 - a child query where the filter is always correct
+cnt := COUNT(ds(assertTrue(seq < 10, 'seq < 10'))) + NOFOLD(100000);
+output(ds(seq != cnt));
+
+
+//Case 2 - a condition that is always false is used from a branch that should never be executed
+cntBad := COUNT(ds(assertTrue(seq > 10, 'seq > 10'))) + NOFOLD(100000);
+
+//NOFOLD is required to stop code generator converting this to a filter (trueValue && ...) which means that cntBad is evaluated always
+//See childds1err.ecl
+cond := IF(trueValue, ds, NOFOLD(ds)(seq != cntBad));
+output(cond);
+

+ 62 - 0
testing/regress/ecl/childds1err.ecl

@@ -0,0 +1,62 @@
+/*##############################################################################
+
+    HPCC SYSTEMS software Copyright (C) 2015 HPCC Systems.
+
+    Licensed 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.
+############################################################################## */
+
+//fail
+//NOTE: This example shouldn't really fail, it illustates a problem converting
+//IF(c, ds, ds(f1)) to ds(c OR f1)
+//which causes dependencies of f1 to be evaluated when they wouldn't be otherwise.
+
+//This is needed to prevent cntBad being hoisted before the resourcing, and becoming unconditional
+//The tests are part of the work aiming to remove this code.
+#option ('workunitTemporaries', false);
+
+idRec := { unsigned id; };
+mainRec := { unsigned seq, dataset(idRec) ids };
+
+idRec createId(unsigned id) := TRANSFORM
+    SELF.id := id;
+END; 
+
+mainRec createMain(unsigned c, unsigned num) := TRANSFORM
+    SELF.seq := c;
+    SELF.ids := DATASET(num, createId(c + (COUNTER-1)));
+END;
+
+ds := NOFOLD(DATASET(4, createMain(COUNTER, 3)));
+
+boolean assertTrue(boolean x, const varstring msg = 'Condition should have been true') := BEGINC++
+    #option pure
+    if (!x)
+        rtlFail(0, msg);
+    return x;
+ENDC++;
+
+trueValue := true : stored('trueValue');
+falseValue := false : stored('falseValue');
+
+//Case 1 - a child query where the filter is always correct
+cnt := COUNT(ds(assertTrue(seq < 10, 'seq < 10'))) + NOFOLD(100000);
+output(ds(seq != cnt));
+
+
+//Case 2 - a condition that is always false is used from a branch that should never be executed
+cntBad := COUNT(ds(assertTrue(seq > 10, 'seq > 10'))) + NOFOLD(100000);
+
+//Problem1: Converting this to a filtered disk read means that cntBad is evaluated always
+cond := IF(trueValue, ds, ds(seq != cntBad));
+output(cond);
+

+ 53 - 0
testing/regress/ecl/childds2.ecl

@@ -0,0 +1,53 @@
+/*##############################################################################
+
+    HPCC SYSTEMS software Copyright (C) 2015 HPCC Systems.
+
+    Licensed 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.
+############################################################################## */
+
+//This is needed to prevent cntBad being hoisted before the resourcing, and becoming unconditional
+//The tests are part of the work aiming to remove this code.
+#option ('workunitTemporaries', false);
+
+idRec := { unsigned id; };
+mainRec := { unsigned seq, dataset(idRec) ids };
+
+idRec createId(unsigned id) := TRANSFORM
+    SELF.id := id;
+END; 
+
+mainRec createMain(unsigned c, unsigned num) := TRANSFORM
+    SELF.seq := c;
+    SELF.ids := DATASET(num, createId(c + (COUNTER-1)));
+END;
+
+ds := NOFOLD(DATASET(4, createMain(COUNTER, 3)));
+
+boolean assertTrue(boolean x, const varstring msg = 'Condition should have been true') := BEGINC++
+    #option pure
+    if (!x)
+        rtlFail(0, msg);
+    return x;
+ENDC++;
+
+trueValue := true : stored('trueValue');
+falseValue := false : stored('falseValue');
+
+//Case 2 - a condition that is always false is used from a multiple branches that should never be executed
+cntBad := COUNT(ds(assertTrue(seq > 10, 'seq > 10'))) + NOFOLD(100000);
+
+//Problem1: Converting this to a filtered disk read means that cntBad is evaluated always
+cond1 := IF(trueValue, ds, NOFOLD(ds)(seq != cntBad));
+cond2 := IF(falseValue, NOFOLD(ds)(seq != cntBad), ds);
+output(cond1+cond2);
+

+ 55 - 0
testing/regress/ecl/childds3.ecl

@@ -0,0 +1,55 @@
+/*##############################################################################
+
+    HPCC SYSTEMS software Copyright (C) 2015 HPCC Systems.
+
+    Licensed 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.
+############################################################################## */
+
+idRec := { unsigned id; };
+mainRec := { unsigned seq, dataset(idRec) ids };
+
+idRec createId(unsigned id) := TRANSFORM
+    SELF.id := id;
+END; 
+
+mainRec createMain(unsigned c, unsigned num) := TRANSFORM
+    SELF.seq := c;
+    SELF.ids := DATASET(num, createId(c + (COUNTER-1)));
+END;
+
+ds := NOFOLD(DATASET(4, createMain(COUNTER, 3)));
+
+boolean assertTrue(boolean x, const varstring msg = 'Condition should have been true') := BEGINC++
+    #option pure
+    if (!x)
+        rtlFail(0, msg);
+    return x;
+ENDC++;
+
+trueValue := true : stored('trueValue');
+falseValue := false : stored('falseValue');
+
+evalFilter(mainRec l) := FUNCTION
+
+    sortedIds := nofold(sort(l.ids, id));
+    cntGood := COUNT(sortedIds(assertTrue(id < 10, 'seq < 10')));
+    RETURN count(sortedIds(id != cntGood)) = 3;
+END;
+
+mainRec t(mainRec l) := TRANSFORM,SKIP(NOT evalFilter(l))
+    SELF := l;
+END;
+
+output(PROJECT(ds, t(LEFT)));
+
+output(ds(evalFilter(ds)));

+ 57 - 0
testing/regress/ecl/childds4.ecl

@@ -0,0 +1,57 @@
+/*##############################################################################
+
+    HPCC SYSTEMS software Copyright (C) 2015 HPCC Systems.
+
+    Licensed 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.
+############################################################################## */
+
+idRec := { unsigned id; };
+mainRec := { unsigned seq, dataset(idRec) ids };
+
+idRec createId(unsigned id) := TRANSFORM
+    SELF.id := id;
+END; 
+
+mainRec createMain(unsigned c, unsigned num) := TRANSFORM
+    SELF.seq := c;
+    SELF.ids := DATASET(num, createId(c + (COUNTER-1)));
+END;
+
+ds := NOFOLD(DATASET(4, createMain(COUNTER, 3)));
+
+boolean assertTrue(boolean x, const varstring msg = 'Condition should have been true') := BEGINC++
+    #option pure
+    if (!x)
+        rtlFail(0, msg);
+    return x;
+ENDC++;
+
+trueValue := true : stored('trueValue');
+falseValue := false : stored('falseValue');
+
+evalFilter(mainRec l) := FUNCTION
+
+    sortedIds := nofold(sort(l.ids, id));
+    cntBad := COUNT(sortedIds(assertTrue(id > 10, 'seq > 10')));
+    
+    f := IF(trueValue, sortedIds, NOFOLD(sortedIds(id != cntBad)));
+    RETURN COUNT(NOFOLD(f)) != 0;
+END;
+
+mainRec t(mainRec l) := TRANSFORM,SKIP(NOT evalFilter(l))
+    SELF := l;
+END;
+
+output(PROJECT(ds, t(LEFT)));
+
+output(ds(evalFilter(ds)));

+ 58 - 0
testing/regress/ecl/childds5.ecl

@@ -0,0 +1,58 @@
+/*##############################################################################
+
+    HPCC SYSTEMS software Copyright (C) 2015 HPCC Systems.
+
+    Licensed 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.
+############################################################################## */
+
+idRec := { unsigned id; };
+mainRec := { unsigned seq, dataset(idRec) ids };
+
+idRec createId(unsigned id) := TRANSFORM
+    SELF.id := id;
+END; 
+
+mainRec createMain(unsigned c, unsigned num) := TRANSFORM
+    SELF.seq := c;
+    SELF.ids := DATASET(num, createId(c + (COUNTER-1)));
+END;
+
+ds := NOFOLD(DATASET(4, createMain(COUNTER, 3)));
+
+boolean assertTrue(boolean x, const varstring msg = 'Condition should have been true') := BEGINC++
+    #option pure
+    if (!x)
+        rtlFail(0, msg);
+    return x;
+ENDC++;
+
+trueValue := true : stored('trueValue');
+falseValue := false : stored('falseValue');
+
+evalFilter(mainRec l) := FUNCTION
+
+    sortedIds := nofold(sort(l.ids, id));
+    cntBad1 := COUNT(sortedIds(assertTrue(id > 10, 'seq > 10')));
+    cntBad2 := COUNT(sortedIds(assertTrue(id > 11, 'seq > 11')));
+    
+    f := IF(trueValue, sortedIds, NOFOLD(sortedIds(id != cntBad1 * cntBad2)));
+    RETURN COUNT(NOFOLD(f)) != 0;
+END;
+
+mainRec t(mainRec l) := TRANSFORM,SKIP(NOT evalFilter(l))
+    SELF := l;
+END;
+
+output(PROJECT(ds, t(LEFT)));
+
+output(ds(evalFilter(ds)));

+ 59 - 0
testing/regress/ecl/childds6.ecl

@@ -0,0 +1,59 @@
+/*##############################################################################
+
+    HPCC SYSTEMS software Copyright (C) 2015 HPCC Systems.
+
+    Licensed 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.
+############################################################################## */
+
+idRec := { unsigned id; };
+mainRec := { unsigned seq, dataset(idRec) ids };
+
+idRec createId(unsigned id) := TRANSFORM
+    SELF.id := id;
+END; 
+
+mainRec createMain(unsigned c, unsigned num) := TRANSFORM
+    SELF.seq := c;
+    SELF.ids := DATASET(num, createId(c + (COUNTER-1)));
+END;
+
+ds := NOFOLD(DATASET(4, createMain(COUNTER, 3)));
+
+boolean assertTrue(boolean x, const varstring msg = 'Condition should have been true') := BEGINC++
+    #option pure
+    if (!x)
+        rtlFail(0, msg);
+    return x;
+ENDC++;
+
+trueValue := true : stored('trueValue');
+falseValue := false : stored('falseValue');
+
+evalFilter(mainRec l) := FUNCTION
+
+    sortedIds := nofold(sort(l.ids, id));
+    bad := sortedIds(assertTrue(id > 10, 'seq > 10'));
+    cntBad1 := COUNT(bad);
+    cntBad2 := SUM(bad, id);
+    
+    f := IF(trueValue, sortedIds, NOFOLD(sortedIds(id != cntBad1 * cntBad2)));
+    RETURN COUNT(NOFOLD(f)) != 0;
+END;
+
+mainRec t(mainRec l) := TRANSFORM,SKIP(NOT evalFilter(l))
+    SELF := l;
+END;
+
+output(PROJECT(ds, t(LEFT)));
+
+output(ds(evalFilter(ds)));

+ 58 - 0
testing/regress/ecl/childds7.ecl

@@ -0,0 +1,58 @@
+/*##############################################################################
+
+    HPCC SYSTEMS software Copyright (C) 2015 HPCC Systems.
+
+    Licensed 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.
+############################################################################## */
+
+idRec := { unsigned id; };
+mainRec := { unsigned seq, dataset(idRec) ids };
+
+idRec createId(unsigned id) := TRANSFORM
+    SELF.id := id;
+END; 
+
+mainRec createMain(unsigned c, unsigned num) := TRANSFORM
+    SELF.seq := c;
+    SELF.ids := DATASET(num, createId(c + (COUNTER-1)));
+END;
+
+ds := NOFOLD(DATASET(4, createMain(COUNTER, 3)));
+
+boolean assertTrue(boolean x, const varstring msg = 'Condition should have been true') := BEGINC++
+    #option pure
+    if (!x)
+        rtlFail(0, msg);
+    return x;
+ENDC++;
+
+trueValue := true : stored('trueValue');
+falseValue := false : stored('falseValue');
+
+evalFilter(mainRec l) := FUNCTION
+
+    dedupIds := DEDUP(l.ids, id);
+    sortedIds := nofold(sort(l.ids, id));
+    bad := sortedIds(assertTrue(id > 10, 'seq > 10'));
+    cntGood := COUNT(sortedIds(id != 10000));
+    cntBad1 := COUNT(bad);
+    
+    f := IF(trueValue, dedupIds(id != cntGood), NOFOLD(dedupIds(id != cntBad1)));
+    RETURN COUNT(NOFOLD(f)) != 0;
+END;
+
+mainRec t(mainRec l) := TRANSFORM,SKIP(NOT evalFilter(l))
+    SELF := l;
+END;
+
+output(PROJECT(ds, t(LEFT)));

+ 58 - 0
testing/regress/ecl/childds7b.ecl

@@ -0,0 +1,58 @@
+/*##############################################################################
+
+    HPCC SYSTEMS software Copyright (C) 2015 HPCC Systems.
+
+    Licensed 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.
+############################################################################## */
+
+idRec := { unsigned id; };
+mainRec := { unsigned seq, dataset(idRec) ids };
+
+idRec createId(unsigned id) := TRANSFORM
+    SELF.id := id;
+END; 
+
+mainRec createMain(unsigned c, unsigned num) := TRANSFORM
+    SELF.seq := c;
+    SELF.ids := DATASET(num, createId(c + (COUNTER-1)));
+END;
+
+ds := NOFOLD(DATASET(4, createMain(COUNTER, 3)));
+
+boolean assertTrue(boolean x, const varstring msg = 'Condition should have been true') := BEGINC++
+    #option pure
+    if (!x)
+        rtlFail(0, msg);
+    return x;
+ENDC++;
+
+trueValue := true : stored('trueValue');
+falseValue := false : stored('falseValue');
+
+evalFilter(mainRec l) := FUNCTION
+
+    dedupIds := l.ids;
+    sortedIds := nofold(sort(l.ids, id));
+    bad := sortedIds(assertTrue(id > 10, 'seq > 10'));
+    cntGood := COUNT(sortedIds(id != 10000));
+    cntBad1 := COUNT(bad);
+    
+    f := IF(trueValue, dedupIds(id != cntGood), NOFOLD(dedupIds(id != cntBad1)));
+    RETURN COUNT(NOFOLD(f)) != 0;
+END;
+
+mainRec t(mainRec l) := TRANSFORM,SKIP(NOT evalFilter(l))
+    SELF := l;
+END;
+
+output(PROJECT(ds, t(LEFT)));

+ 12 - 0
testing/regress/ecl/key/childds1.xml

@@ -0,0 +1,12 @@
+<Dataset name='Result 1'>
+ <Row><seq>1</seq><ids><Row><id>1</id></Row><Row><id>2</id></Row><Row><id>3</id></Row></ids></Row>
+ <Row><seq>2</seq><ids><Row><id>2</id></Row><Row><id>3</id></Row><Row><id>4</id></Row></ids></Row>
+ <Row><seq>3</seq><ids><Row><id>3</id></Row><Row><id>4</id></Row><Row><id>5</id></Row></ids></Row>
+ <Row><seq>4</seq><ids><Row><id>4</id></Row><Row><id>5</id></Row><Row><id>6</id></Row></ids></Row>
+</Dataset>
+<Dataset name='Result 2'>
+ <Row><seq>1</seq><ids><Row><id>1</id></Row><Row><id>2</id></Row><Row><id>3</id></Row></ids></Row>
+ <Row><seq>2</seq><ids><Row><id>2</id></Row><Row><id>3</id></Row><Row><id>4</id></Row></ids></Row>
+ <Row><seq>3</seq><ids><Row><id>3</id></Row><Row><id>4</id></Row><Row><id>5</id></Row></ids></Row>
+ <Row><seq>4</seq><ids><Row><id>4</id></Row><Row><id>5</id></Row><Row><id>6</id></Row></ids></Row>
+</Dataset>

+ 1 - 0
testing/regress/ecl/key/childds1err.xml

@@ -0,0 +1 @@
+<Exception><Source>eclagent</Source><Message>System error: 0: Graph[4], filter[7]: SLAVE #1 [192.168.0.200:20100]: seq &gt; 10, </Message></Exception>

+ 10 - 0
testing/regress/ecl/key/childds2.xml

@@ -0,0 +1,10 @@
+<Dataset name='Result 1'>
+ <Row><seq>1</seq><ids><Row><id>1</id></Row><Row><id>2</id></Row><Row><id>3</id></Row></ids></Row>
+ <Row><seq>2</seq><ids><Row><id>2</id></Row><Row><id>3</id></Row><Row><id>4</id></Row></ids></Row>
+ <Row><seq>3</seq><ids><Row><id>3</id></Row><Row><id>4</id></Row><Row><id>5</id></Row></ids></Row>
+ <Row><seq>4</seq><ids><Row><id>4</id></Row><Row><id>5</id></Row><Row><id>6</id></Row></ids></Row>
+ <Row><seq>1</seq><ids><Row><id>1</id></Row><Row><id>2</id></Row><Row><id>3</id></Row></ids></Row>
+ <Row><seq>2</seq><ids><Row><id>2</id></Row><Row><id>3</id></Row><Row><id>4</id></Row></ids></Row>
+ <Row><seq>3</seq><ids><Row><id>3</id></Row><Row><id>4</id></Row><Row><id>5</id></Row></ids></Row>
+ <Row><seq>4</seq><ids><Row><id>4</id></Row><Row><id>5</id></Row><Row><id>6</id></Row></ids></Row>
+</Dataset>

+ 6 - 0
testing/regress/ecl/key/childds3.xml

@@ -0,0 +1,6 @@
+<Dataset name='Result 1'>
+ <Row><seq>4</seq><ids><Row><id>4</id></Row><Row><id>5</id></Row><Row><id>6</id></Row></ids></Row>
+</Dataset>
+<Dataset name='Result 2'>
+ <Row><seq>4</seq><ids><Row><id>4</id></Row><Row><id>5</id></Row><Row><id>6</id></Row></ids></Row>
+</Dataset>

+ 12 - 0
testing/regress/ecl/key/childds4.xml

@@ -0,0 +1,12 @@
+<Dataset name='Result 1'>
+ <Row><seq>1</seq><ids><Row><id>1</id></Row><Row><id>2</id></Row><Row><id>3</id></Row></ids></Row>
+ <Row><seq>2</seq><ids><Row><id>2</id></Row><Row><id>3</id></Row><Row><id>4</id></Row></ids></Row>
+ <Row><seq>3</seq><ids><Row><id>3</id></Row><Row><id>4</id></Row><Row><id>5</id></Row></ids></Row>
+ <Row><seq>4</seq><ids><Row><id>4</id></Row><Row><id>5</id></Row><Row><id>6</id></Row></ids></Row>
+</Dataset>
+<Dataset name='Result 2'>
+ <Row><seq>1</seq><ids><Row><id>1</id></Row><Row><id>2</id></Row><Row><id>3</id></Row></ids></Row>
+ <Row><seq>2</seq><ids><Row><id>2</id></Row><Row><id>3</id></Row><Row><id>4</id></Row></ids></Row>
+ <Row><seq>3</seq><ids><Row><id>3</id></Row><Row><id>4</id></Row><Row><id>5</id></Row></ids></Row>
+ <Row><seq>4</seq><ids><Row><id>4</id></Row><Row><id>5</id></Row><Row><id>6</id></Row></ids></Row>
+</Dataset>

+ 12 - 0
testing/regress/ecl/key/childds5.xml

@@ -0,0 +1,12 @@
+<Dataset name='Result 1'>
+ <Row><seq>1</seq><ids><Row><id>1</id></Row><Row><id>2</id></Row><Row><id>3</id></Row></ids></Row>
+ <Row><seq>2</seq><ids><Row><id>2</id></Row><Row><id>3</id></Row><Row><id>4</id></Row></ids></Row>
+ <Row><seq>3</seq><ids><Row><id>3</id></Row><Row><id>4</id></Row><Row><id>5</id></Row></ids></Row>
+ <Row><seq>4</seq><ids><Row><id>4</id></Row><Row><id>5</id></Row><Row><id>6</id></Row></ids></Row>
+</Dataset>
+<Dataset name='Result 2'>
+ <Row><seq>1</seq><ids><Row><id>1</id></Row><Row><id>2</id></Row><Row><id>3</id></Row></ids></Row>
+ <Row><seq>2</seq><ids><Row><id>2</id></Row><Row><id>3</id></Row><Row><id>4</id></Row></ids></Row>
+ <Row><seq>3</seq><ids><Row><id>3</id></Row><Row><id>4</id></Row><Row><id>5</id></Row></ids></Row>
+ <Row><seq>4</seq><ids><Row><id>4</id></Row><Row><id>5</id></Row><Row><id>6</id></Row></ids></Row>
+</Dataset>

+ 12 - 0
testing/regress/ecl/key/childds6.xml

@@ -0,0 +1,12 @@
+<Dataset name='Result 1'>
+ <Row><seq>1</seq><ids><Row><id>1</id></Row><Row><id>2</id></Row><Row><id>3</id></Row></ids></Row>
+ <Row><seq>2</seq><ids><Row><id>2</id></Row><Row><id>3</id></Row><Row><id>4</id></Row></ids></Row>
+ <Row><seq>3</seq><ids><Row><id>3</id></Row><Row><id>4</id></Row><Row><id>5</id></Row></ids></Row>
+ <Row><seq>4</seq><ids><Row><id>4</id></Row><Row><id>5</id></Row><Row><id>6</id></Row></ids></Row>
+</Dataset>
+<Dataset name='Result 2'>
+ <Row><seq>1</seq><ids><Row><id>1</id></Row><Row><id>2</id></Row><Row><id>3</id></Row></ids></Row>
+ <Row><seq>2</seq><ids><Row><id>2</id></Row><Row><id>3</id></Row><Row><id>4</id></Row></ids></Row>
+ <Row><seq>3</seq><ids><Row><id>3</id></Row><Row><id>4</id></Row><Row><id>5</id></Row></ids></Row>
+ <Row><seq>4</seq><ids><Row><id>4</id></Row><Row><id>5</id></Row><Row><id>6</id></Row></ids></Row>
+</Dataset>

+ 6 - 0
testing/regress/ecl/key/childds7.xml

@@ -0,0 +1,6 @@
+<Dataset name='Result 1'>
+ <Row><seq>1</seq><ids><Row><id>1</id></Row><Row><id>2</id></Row><Row><id>3</id></Row></ids></Row>
+ <Row><seq>2</seq><ids><Row><id>2</id></Row><Row><id>3</id></Row><Row><id>4</id></Row></ids></Row>
+ <Row><seq>3</seq><ids><Row><id>3</id></Row><Row><id>4</id></Row><Row><id>5</id></Row></ids></Row>
+ <Row><seq>4</seq><ids><Row><id>4</id></Row><Row><id>5</id></Row><Row><id>6</id></Row></ids></Row>
+</Dataset>

+ 6 - 0
testing/regress/ecl/key/childds7b.xml

@@ -0,0 +1,6 @@
+<Dataset name='Result 1'>
+ <Row><seq>1</seq><ids><Row><id>1</id></Row><Row><id>2</id></Row><Row><id>3</id></Row></ids></Row>
+ <Row><seq>2</seq><ids><Row><id>2</id></Row><Row><id>3</id></Row><Row><id>4</id></Row></ids></Row>
+ <Row><seq>3</seq><ids><Row><id>3</id></Row><Row><id>4</id></Row><Row><id>5</id></Row></ids></Row>
+ <Row><seq>4</seq><ids><Row><id>4</id></Row><Row><id>5</id></Row><Row><id>6</id></Row></ids></Row>
+</Dataset>