瀏覽代碼

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 年之前
父節點
當前提交
fa5bdebbbd

+ 4 - 0
ecl/hql/hqlatoms.cpp

@@ -189,6 +189,7 @@ IAtom * gctxmethodAtom;
 IAtom * getAtom;
 IAtom * globalAtom;
 IAtom * graphAtom;
+IAtom * _graphLocal_Atom;
 IAtom * groupAtom;
 IAtom * groupedAtom;
 IAtom * hashAtom;
@@ -226,6 +227,7 @@ IAtom * keyedAtom;
 IAtom * labeledAtom;
 IAtom * languageAtom;
 IAtom * lastAtom;
+IAtom * _lazy_Atom;
 IAtom * leftAtom;
 IAtom * leftonlyAtom;
 IAtom * leftouterAtom;
@@ -622,6 +624,7 @@ MODULE_INIT(INIT_PRIORITY_HQLATOM)
     MAKEATOM(global);
     MAKEATOM(globalContext);
     MAKEATOM(graph);
+    MAKESYSATOM(graphLocal);
     MAKEATOM(group);
     MAKEATOM(grouped);
     MAKEATOM(hash);
@@ -660,6 +663,7 @@ MODULE_INIT(INIT_PRIORITY_HQLATOM)
     MAKEATOM(labeled);
     MAKEATOM(language);
     MAKEATOM(last);
+    MAKESYSATOM(lazy);
     MAKEATOM(left);
     leftonlyAtom = createLowerCaseAtom("left only");
     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 * languageAtom;
 extern HQL_API IAtom * lastAtom;
+extern HQL_API IAtom * _lazy_Atom;
 extern HQL_API IAtom * leftAtom;
 extern HQL_API IAtom * leftonlyAtom;
 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)
 {
     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 bool isCommonSubstringRange(IHqlExpression * expr);
+extern HQL_API bool isFileOutput(IHqlExpression * expr);
+extern HQL_API bool isWorkunitOutput(IHqlExpression * expr);
 
 class HQL_API AtmostLimit
 {

+ 0 - 2
ecl/hqlcpp/hqlcatom.cpp

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

+ 0 - 1
ecl/hqlcpp/hqlcatom.hpp

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

+ 1 - 0
ecl/hqlcpp/hqlcpp.cpp

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

+ 1 - 0
ecl/hqlcpp/hqlcpp.ipp

@@ -746,6 +746,7 @@ struct HqlCppOptions
     bool                expandSelectCreateRow;
     bool                optimizeSortAllFields;
     bool                optimizeSortAllFieldsStrict;
+    bool                alwaysReuseGlobalSpills;
 };
 
 //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);
             break;
         case no_output:
-            return (queryRealChild(expr, 1) != NULL);
+            return isFileOutput(expr);
         case no_actionlist:
         case no_orderedactionlist:
             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
 {
 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 setChildQuery(bool value);
+    void setNewChildQuery(IHqlExpression * graphIdExpr, unsigned numResults);
+    void setUseGraphResults(bool _useGraphResults)
+    {
+        useGraphResults = _useGraphResults;
+    }
+    bool useGraphResult(bool linkedFromChild);
+    bool useGlobalResult(bool linkedFromChild);
 
 public:
     UniqueSequenceCounter & spillSequence;
@@ -105,6 +82,9 @@ public:
     bool     useResultsForChildSpills;
     bool     alwaysUseGraphResults;
     bool     newBalancedSpotter;
+    bool     spillMultiCondition;
+    bool     spotThroughAggregate;
+    bool     alwaysReuseGlobalSpills;
 
     IHqlExpression * graphIdExpr;
     unsigned nextResult;
@@ -160,7 +140,8 @@ public:
     virtual void changeSourceGraph(ResourceGraphInfo * newGraph);
     virtual void changeSinkGraph(ResourceGraphInfo * newGraph);
     virtual bool isRedundantLink();
-    virtual bool isDependency() { return false; }
+    virtual bool isDependency() const { return false; }
+    virtual IHqlExpression * queryDependency() const { return NULL; }
 
 protected:
     void trace(const char * name);
@@ -176,12 +157,16 @@ public:
 class ResourceGraphDependencyLink : public ResourceGraphLink
 {
 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 changeSinkGraph(ResourceGraphInfo * newGraph);
     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;
@@ -204,7 +189,7 @@ public:
     bool isDependentOn(ResourceGraphInfo & other, bool allowDirect);
     bool isVeryCheap();
     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);
 
     bool isSharedInput(IHqlExpression * expr);
@@ -327,14 +312,40 @@ public:
     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:
     enum { PathUnknown, PathConditional, PathUnconditional };
 
     ResourcerInfo(IHqlExpression * _original, CResourceOptions * _options);
 
-    IHqlExpression * createSpilledRead(IHqlExpression * spillReason);
     IHqlExpression * createTransformedExpr(IHqlExpression * expr);
 
     bool addCondition(IHqlExpression * condition);
@@ -356,8 +367,12 @@ public:
     void resetBalanced();
     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 isConditionExpr()
     {
@@ -393,34 +408,20 @@ public:
 
 protected:
     bool spillSharesSplitter();
-    bool useGraphResult();
-    bool useGlobalResult();
     IHqlExpression * createAggregation(IHqlExpression * expr);
-    IHqlExpression * createSpilledWrite(IHqlExpression * transformed);
     IHqlExpression * createSpiller(IHqlExpression * transformed, bool reuseSplitter);
     IHqlExpression * createSplitter(IHqlExpression * transformed);
 
-protected:
-    void addSpillFlags(HqlExprArray & args, bool isRead);
-    IHqlExpression * createSpillName();
-    IHqlExpression * wrapRowOwn(IHqlExpression * expr);
-
 public:
-    HqlExprAttr original;
     Owned<ResourceGraphInfo> graph;
-    HqlExprAttr spillName;
-    IHqlExpression * transformed;
-    IHqlExpression * outputToUseForSpill;
-    CResourceOptions * options;
     HqlExprAttr pathToSplitter;
     HqlExprArray aggregates;
     HqlExprArray conditions;
-    ChildDependentArray childDependents;
-    HqlExprAttr spilledDataset;
     HqlExprAttr splitterOutput;
     HqlExprArray projected;
     HqlExprAttr projectedExpr;
     CIArrayOf<CSplitterLink> balancedLinks;
+    GraphLinkArray dependsOn;           // NB: These do no link....
 
     unsigned numUses;
     unsigned numExternalUses;
@@ -440,7 +441,6 @@ public:
     bool isSpillPoint:1;
     bool balanced:1;
     bool isAlreadyInScope:1;
-    bool linkedFromChild:1;
     bool forceHoist:1;
     bool neverSplit:1;
     bool isConditionalFilter:1;
@@ -449,32 +449,19 @@ public:
     bool balancedVisiting:1;
 };
 
-struct DependencySourceInfo
-{
-    HqlExprArray                    search;
-    CIArrayOf<ResourceGraphInfo>    graphs;
-    HqlExprArray                    exprs;
-};
-
-
+class EclResourceDependencyGatherer;
 class EclResourcer
 {
     friend class SelectHoistTransformer;
     friend class CSplitterInfo;
 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();
 
     void resourceGraph(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 setUseGraphResults(bool _useGraphResults) 
-    { 
-        options.useGraphResults = _useGraphResults; 
-    }
-    void tagActiveCursors(HqlExprCopyArray & activeRows);
+    void tagActiveCursors(HqlExprCopyArray * activeRows);
     inline unsigned numGraphResults() { return options.nextResult; }
 
 protected:
@@ -487,15 +474,11 @@ protected:
     void replaceGraphReferences(ResourceGraphInfo * oldGraph, ResourceGraphInfo * newGraph);
 
 //Pass 1
-    void gatherChildSplitPoints(IHqlExpression * expr, ResourcerInfo * info, unsigned first, unsigned last);
     bool findSplitPoints(IHqlExpression * expr, bool isProjected);
     void findSplitPoints(HqlExprArray & exprs);
     void noteConditionalChildren(BoolArray & alwaysHoistChild);
     void deriveUsageCounts(IHqlExpression * expr);
     void deriveUsageCounts(const HqlExprArray & exprs);
-    void extendSplitPoints();
-    void projectChildDependents();
-    IHqlExpression * projectChildDependent(IHqlExpression * expr);
 
 //Pass 2
     void createInitialGraph(IHqlExpression * expr, IHqlExpression * owner, ResourceGraphInfo * ownerGraph, LinkKind linkKind, bool forceNewGraph);
@@ -520,13 +503,7 @@ protected:
     void resourceSubGraphs(HqlExprArray & exprs);
 
 //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);
 
 //Pass 6
@@ -560,7 +537,6 @@ protected:
     void moveExternalSpillPoints();
 
 //Pass 9
-    IHqlExpression * replaceResourcedReferences(ResourcerInfo * info, IHqlExpression * expr);
     IHqlExpression * doCreateResourced(IHqlExpression * expr, ResourceGraphInfo * graph, bool expandInParent, bool defineSideEffect);
     IHqlExpression * createResourced(IHqlExpression * expr, ResourceGraphInfo * graph, bool expandInParent, bool defineSideEffect);
     void createResourced(HqlExprArray & transformed);
@@ -582,21 +558,16 @@ protected:
     CIArrayOf<ResourceGraphInfo> graphs;
     CIArrayOf<ResourceGraphLink> links;
     ClusterType targetClusterType;
-    DependencySourceInfo dependencySource;                  
-    unsigned clusterSize;
     CResources * resourceLimit;
     IErrorReceiver * errors;
     unsigned thisPass;
     bool spilled;
-    bool spillMultiCondition;
-    bool spotThroughAggregate;
     bool insideNeverSplit;
     bool insideSteppedNeverSplit;
     bool sequential;
-    CResourceOptions options;
+    CResourceOptions & options;
     HqlExprArray rootConditions;
     HqlExprCopyArray activeSelectors;
-    ChildDependentArray allChildDependents;
 };
 
 #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>