From 99bd5ec02290e729b010d74dbd05a566899de4da Mon Sep 17 00:00:00 2001 From: morrySnow <101034200+morrySnow@users.noreply.github.com> Date: Thu, 30 Mar 2023 16:09:16 +0800 Subject: [PATCH] [fix](Nereids) fix some bugs in Subquery to window rule (#18233) we introduce this rule by PR #17968, but some corner case do not be processed correctly. This PR fix these bugs: 1. fix window function generation method, replace inner slot with equivalent outer slot 2. forbid below scenes a. inner has a mapping project b. inner has an unexpected filter c. outer has a mapping project d. outer has an unexpected filter e. outer has additional table f. outer has same table g. outer and inner with different join condition h. outer and inner has same table with different join condition --- .../AggScalarSubQueryToWindowFunction.java | 359 +- .../trees/expressions/functions/agg/Avg.java | 3 +- .../expressions/functions/agg/Count.java | 3 +- .../trees/expressions/functions/agg/Max.java | 4 +- .../trees/expressions/functions/agg/Min.java | 4 +- .../trees/expressions/functions/agg/Sum.java | 3 +- .../window/SupportWindowAnalytic.java | 24 + .../functions/window/WindowFunction.java | 2 +- ...AggScalarSubQueryToWindowFunctionTest.java | 186 +- .../data/nereids_tpch_p0/tpch/q1.out | 16 +- .../data/nereids_tpch_p0/tpch/q10.out | 80 +- .../data/nereids_tpch_p0/tpch/q11.out | 12424 ++++++++-------- .../data/nereids_tpch_p0/tpch/q14.out | 4 +- .../data/nereids_tpch_p0/tpch/q15.out | 4 +- .../data/nereids_tpch_p0/tpch/q17.out | 4 +- .../data/nereids_tpch_p0/tpch/q18.out | 20 +- .../data/nereids_tpch_p0/tpch/q19.out | 4 +- .../data/nereids_tpch_p0/tpch/q2.out | 176 +- .../data/nereids_tpch_p0/tpch/q22.out | 28 +- .../data/nereids_tpch_p0/tpch/q3.out | 40 +- .../data/nereids_tpch_p0/tpch/q5.out | 20 +- .../data/nereids_tpch_p0/tpch/q6.out | 4 +- .../data/nereids_tpch_p0/tpch/q7.out | 16 +- .../data/nereids_tpch_p0/tpch/q8.out | 8 +- .../data/nereids_tpch_p0/tpch/q9.out | 700 +- .../suites/nereids_tpch_p0/ddl/customer.sql | 2 +- .../suites/nereids_tpch_p0/ddl/lineitem.sql | 8 +- .../suites/nereids_tpch_p0/ddl/orders.sql | 2 +- .../suites/nereids_tpch_p0/ddl/part.sql | 2 +- .../suites/nereids_tpch_p0/ddl/partsupp.sql | 2 +- .../suites/nereids_tpch_p0/ddl/supplier.sql | 2 +- 31 files changed, 7197 insertions(+), 6957 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/SupportWindowAnalytic.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/AggScalarSubQueryToWindowFunction.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/AggScalarSubQueryToWindowFunction.java index 2603ca611c..d06da96f7d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/AggScalarSubQueryToWindowFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/logical/AggScalarSubQueryToWindowFunction.java @@ -20,35 +20,28 @@ package org.apache.doris.nereids.rules.rewrite.logical; import org.apache.doris.nereids.jobs.JobContext; import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.ComparisonPredicate; -import org.apache.doris.nereids.trees.expressions.EqualTo; import org.apache.doris.nereids.trees.expressions.ExprId; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.expressions.WindowExpression; import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; -import org.apache.doris.nereids.trees.expressions.functions.agg.Avg; -import org.apache.doris.nereids.trees.expressions.functions.agg.Count; -import org.apache.doris.nereids.trees.expressions.functions.agg.Max; -import org.apache.doris.nereids.trees.expressions.functions.agg.Min; import org.apache.doris.nereids.trees.expressions.functions.agg.NullableAggregateFunction; -import org.apache.doris.nereids.trees.expressions.functions.agg.Sum; +import org.apache.doris.nereids.trees.expressions.functions.window.SupportWindowAnalytic; import org.apache.doris.nereids.trees.expressions.literal.Literal; -import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionRewriter; import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionVisitor; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; import org.apache.doris.nereids.trees.plans.logical.LogicalApply; import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; -import org.apache.doris.nereids.trees.plans.logical.LogicalLimit; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; import org.apache.doris.nereids.trees.plans.logical.LogicalRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalWindow; import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanRewriter; -import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanVisitor; import org.apache.doris.nereids.util.ExpressionUtils; import org.apache.doris.nereids.util.PlanUtils; @@ -57,6 +50,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.collect.Sets; import java.util.Collection; @@ -64,10 +58,8 @@ import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Optional; import java.util.Set; -import java.util.function.Function; import java.util.stream.Collectors; /** @@ -75,99 +67,185 @@ import java.util.stream.Collectors; * logicalFilter(logicalApply(any(), logicalAggregate())) * to * logicalProject((logicalFilter(logicalWindow(logicalFilter(any()))))) + *

* refer paper: WinMagic - Subquery Elimination Using Window Aggregation + *

+ * TODO: use materialized view pattern match to do outer and inner tree match. */ public class AggScalarSubQueryToWindowFunction extends DefaultPlanRewriter implements CustomRewriter { - private static final Set> SUPPORTED_FUNCTION = ImmutableSet.of( - Min.class, Max.class, Count.class, Sum.class, Avg.class - ); - private static final Set> LEFT_SUPPORTED_PLAN = ImmutableSet.of( - LogicalRelation.class, LogicalJoin.class, LogicalProject.class, LogicalFilter.class, LogicalLimit.class - ); - private static final Set> RIGHT_SUPPORTED_PLAN = ImmutableSet.of( - LogicalRelation.class, LogicalJoin.class, LogicalProject.class, LogicalFilter.class, LogicalAggregate.class - ); - private List outerPlans = null; - private List innerPlans = null; - private LogicalAggregate aggOp = null; - private List functions = null; + private static final Set> OUTER_SUPPORTED_PLAN = ImmutableSet.of( + LogicalJoin.class, + LogicalProject.class, + LogicalRelation.class + ); + + private static final Set> INNER_SUPPORTED_PLAN = ImmutableSet.of( + LogicalAggregate.class, + LogicalFilter.class, + LogicalJoin.class, + LogicalProject.class, + LogicalRelation.class + ); + + private final List outerPlans = Lists.newArrayList(); + private final List innerPlans = Lists.newArrayList(); + private final List functions = Lists.newArrayList(); + private final Map innerOuterSlotMap = Maps.newHashMap(); + + /** + * the entrance of this rule. we only override one visitor: visitLogicalFilter + * because we need to process the filter of outer plan. It is on the top of Apply. + */ @Override public Plan rewriteRoot(Plan plan, JobContext context) { return plan.accept(this, context); } + /** + * we need to process Filter and Apply, but sometimes there are project between Filter and Apply. + * According to {@link org.apache.doris.nereids.rules.analysis.SubqueryToApply} rule. The project + * is used to project apply output to original output, it is not affect this rule at all. so we ignore it. + */ @Override public Plan visitLogicalFilter(LogicalFilter filter, JobContext context) { - LogicalApply> apply = checkPattern(filter); - if (apply == null) { - return filter; - } - if (!check(filter, apply)) { - return filter; - } - return trans(filter, apply); + return findApply(filter) + .filter(a -> check(filter, a)) + .map(a -> rewrite(filter, a)) + .orElse(filter); } - private LogicalApply> checkPattern(LogicalFilter filter) { - LogicalPlan plan = ((LogicalPlan) filter.child()); - if (plan instanceof LogicalProject) { - plan = ((LogicalPlan) ((LogicalProject) plan).child()); - } - if (!(plan instanceof LogicalApply)) { - return null; - } - LogicalApply apply = (LogicalApply) plan; - if (!checkApplyNode(apply)) { - return null; - } - return apply.right() instanceof LogicalAggregate ? apply : null; + private Optional> findApply(LogicalFilter filter) { + return Optional.of(filter.child()) + .map(p -> p instanceof LogicalProject ? p.child(0) : p) + .filter(LogicalApply.class::isInstance) + .map(p -> (LogicalApply) p); } - private boolean check(LogicalFilter filter, LogicalApply> apply) { - LogicalPlan outer = ((LogicalPlan) apply.child(0)); - LogicalPlan inner = ((LogicalPlan) apply.child(1)); - outerPlans = PlanCollector.INSTANCE.collect(outer); - innerPlans = PlanCollector.INSTANCE.collect(inner); - Optional innerFilter = innerPlans.stream() - .filter(LogicalFilter.class::isInstance) - .map(LogicalFilter.class::cast).findFirst(); - return innerFilter.isPresent() - && checkPlanType() && checkAggType() + private boolean check(LogicalFilter outerFilter, LogicalApply apply) { + outerPlans.addAll(apply.child(0).collect(LogicalPlan.class::isInstance)); + innerPlans.addAll(apply.child(1).collect(LogicalPlan.class::isInstance)); + + return checkPlanType() + && checkApply(apply) + && checkAggregate() + && checkJoin() + && checkProject() && checkRelation(apply.getCorrelationSlot()) - && checkPredicate(Sets.newHashSet(filter.getConjuncts()), - Sets.newHashSet(innerFilter.get().getConjuncts())); + && checkFilter(outerFilter); } // check children's nodes because query process will be changed private boolean checkPlanType() { - return outerPlans.stream().allMatch(p -> LEFT_SUPPORTED_PLAN.stream().anyMatch(c -> c.isInstance(p))) - && innerPlans.stream().allMatch(p -> RIGHT_SUPPORTED_PLAN.stream().anyMatch(c -> c.isInstance(p))); + return outerPlans.stream().allMatch(p -> OUTER_SUPPORTED_PLAN.stream().anyMatch(c -> c.isInstance(p))) + && innerPlans.stream().allMatch(p -> INNER_SUPPORTED_PLAN.stream().anyMatch(c -> c.isInstance(p))); } - private boolean checkApplyNode(LogicalApply apply) { - return apply.isScalar() && apply.isCorrelated() && apply.getSubCorrespondingConjunct().isPresent() + /** + * Apply should be + * 1. scalar + * 2. is not mark join + * 3. is correlated + * 4. correlated conjunct should be {@link ComparisonPredicate} + * 5. the top plan of Apply inner should be {@link LogicalAggregate} + */ + private boolean checkApply(LogicalApply apply) { + return apply.isScalar() + && !apply.isMarkJoin() + && apply.right() instanceof LogicalAggregate + && apply.isCorrelated() + && apply.getSubCorrespondingConjunct().isPresent() && apply.getSubCorrespondingConjunct().get() instanceof ComparisonPredicate; } - // check aggregation of inner scope - private boolean checkAggType() { - List aggSet = innerPlans.stream().filter(LogicalAggregate.class::isInstance) - .map(LogicalAggregate.class::cast) + /** + * check aggregation of inner scope, it should be only one Aggregate and only one AggregateFunction in it + */ + private boolean checkAggregate() { + List> aggSet = innerPlans.stream().filter(LogicalAggregate.class::isInstance) + .map(p -> (LogicalAggregate) p) .collect(Collectors.toList()); - if (aggSet.size() > 1) { + if (aggSet.size() != 1) { // window functions don't support nesting. return false; } - aggOp = aggSet.get(0); - functions = ((List) ExpressionUtils.collectAll( + LogicalAggregate aggOp = aggSet.get(0); + functions.addAll(ExpressionUtils.collectAll( aggOp.getOutputExpressions(), AggregateFunction.class::isInstance)); - Preconditions.checkArgument(functions.size() == 1); - return functions.stream().allMatch(f -> SUPPORTED_FUNCTION.contains(f.getClass()) && !f.isDistinct()); + if (functions.size() != 1) { + return false; + } + return functions.stream().allMatch(f -> f instanceof SupportWindowAnalytic && !f.isDistinct()); } - // check if the relations of the outer's includes the inner's + /** + * check inner scope only have one filter. and inner filter is a sub collection of outer filter + */ + private boolean checkFilter(LogicalFilter outerFilter) { + List> innerFilters = innerPlans.stream() + .filter(LogicalFilter.class::isInstance) + .map(p -> (LogicalFilter) p).collect(Collectors.toList()); + if (innerFilters.size() != 1) { + return false; + } + Set outerConjunctSet = Sets.newHashSet(outerFilter.getConjuncts()); + Set innerConjunctSet = innerFilters.get(0).getConjuncts().stream() + .map(e -> ExpressionUtils.replace(e, innerOuterSlotMap)) + .collect(Collectors.toSet()); + Iterator innerIterator = innerConjunctSet.iterator(); + // inner predicate should be the sub-set of outer predicate. + while (innerIterator.hasNext()) { + Expression innerExpr = innerIterator.next(); + Iterator outerIterator = outerConjunctSet.iterator(); + while (outerIterator.hasNext()) { + Expression outerExpr = outerIterator.next(); + if (ExpressionIdenticalChecker.INSTANCE.check(innerExpr, outerExpr)) { + innerIterator.remove(); + outerIterator.remove(); + } + } + } + // now the expressions are all like 'expr op literal' or flipped, and whose expr is not correlated. + return innerConjunctSet.isEmpty(); + } + + /** + * check join to ensure no condition on it. + * this is because we cannot do accurate pattern match between outer scope and inner scope + * so, we currently forbid join with condition here. + */ + private boolean checkJoin() { + return outerPlans.stream() + .filter(LogicalJoin.class::isInstance) + .map(p -> (LogicalJoin) p) + .noneMatch(j -> j.getOnClauseCondition().isPresent()) + && innerPlans.stream() + .filter(LogicalJoin.class::isInstance) + .map(p -> (LogicalJoin) p) + .noneMatch(j -> j.getOnClauseCondition().isPresent()); + } + + /** + * check inner and outer project to ensure no project except column pruning + */ + private boolean checkProject() { + return outerPlans.stream() + .filter(LogicalProject.class::isInstance) + .map(p -> (LogicalProject) p) + .allMatch(p -> p.getExpressions().stream().allMatch(SlotReference.class::isInstance)) + && innerPlans.stream() + .filter(LogicalProject.class::isInstance) + .map(p -> (LogicalProject) p) + .allMatch(p -> p.getExpressions().stream().allMatch(SlotReference.class::isInstance)); + } + + /** + * check inner and outer relation + * 1. outer table size - inner table size must equal to 1 + * 2. outer table list - inner table list should only remain 1 table + * 3. the remaining table in step 2 should be correlated table for inner plan + */ private boolean checkRelation(List correlatedSlots) { List outerTables = outerPlans.stream().filter(LogicalRelation.class::isInstance) .map(LogicalRelation.class::cast) @@ -176,15 +254,21 @@ public class AggScalarSubQueryToWindowFunction extends DefaultPlanRewriter outerIds = outerTables.stream().map(node -> node.getTable().getId()).collect(Collectors.toSet()); - Set innerIds = innerTables.stream().map(node -> node.getTable().getId()).collect(Collectors.toSet()); - - Set outerCopy = Sets.newHashSet(outerIds); - outerIds.removeAll(innerIds); - innerIds.removeAll(outerCopy); - if (outerIds.isEmpty() || !innerIds.isEmpty()) { + List outerIds = outerTables.stream().map(node -> node.getTable().getId()).collect(Collectors.toList()); + List innerIds = innerTables.stream().map(node -> node.getTable().getId()).collect(Collectors.toList()); + if (Sets.newHashSet(outerIds).size() != outerIds.size() + || Sets.newHashSet(innerIds).size() != innerIds.size()) { return false; } + if (outerIds.size() - innerIds.size() != 1) { + return false; + } + innerIds.forEach(outerIds::remove); + if (outerIds.size() != 1) { + return false; + } + + createSlotMapping(outerTables, innerTables); Set correlatedRelationOutput = outerTables.stream() .filter(node -> outerIds.contains(node.getTable().getId())) @@ -194,26 +278,28 @@ public class AggScalarSubQueryToWindowFunction extends DefaultPlanRewriter correlatedRelationOutput.contains(e.getExprId())); } - private boolean checkPredicate(Set outerConjuncts, Set innerConjuncts) { - Iterator innerIter = innerConjuncts.iterator(); - // inner predicate should be the sub-set of outer predicate. - while (innerIter.hasNext()) { - Expression innerExpr = innerIter.next(); - Iterator outerIter = outerConjuncts.iterator(); - while (outerIter.hasNext()) { - Expression outerExpr = outerIter.next(); - if (ExpressionIdenticalChecker.INSTANCE.check(innerExpr, outerExpr)) { - innerIter.remove(); - outerIter.remove(); + private void createSlotMapping(List outerTables, List innerTables) { + for (LogicalRelation outerTable : outerTables) { + for (LogicalRelation innerTable : innerTables) { + if (innerTable.getTable().getId() == outerTable.getTable().getId()) { + for (Slot innerSlot : innerTable.getOutput()) { + for (Slot outerSlot : outerTable.getOutput()) { + if (innerSlot.getName().equals(outerSlot.getName())) { + innerOuterSlotMap.put(innerSlot, outerSlot); + break; + } + } + } + break; } } } - // now the expressions are all like 'expr op literal' or flipped, and whose expr is not correlated. - return innerConjuncts.size() == 0; } - private Plan trans(LogicalFilter filter, LogicalApply> apply) { - LogicalAggregate agg = apply.right(); + private Plan rewrite(LogicalFilter filter, LogicalApply apply) { + Preconditions.checkArgument(apply.right() instanceof LogicalAggregate, + "right child of Apply should be LogicalAggregate"); + LogicalAggregate agg = (LogicalAggregate) apply.right(); // transform algorithm // first: find the slot in outer scope corresponding to the slot in aggregate function in inner scope. @@ -228,7 +314,7 @@ public class AggScalarSubQueryToWindowFunction extends DefaultPlanRewriter windowAggSlots = windowFilterConjunct.child(0).collectToList(Slot.class::isInstance); - AggregateFunction function = functions.get(0); if (function instanceof NullableAggregateFunction) { // adjust agg function's nullable. @@ -251,7 +334,7 @@ public class AggScalarSubQueryToWindowFunction extends DefaultPlanRewriter windowFunctionAlias.toSlot())); + aggOutExpr = ExpressionUtils.replace(aggOutExpr, ImmutableMap + .of(functions.get(0), windowFunctionAlias.toSlot())); // we change the child contains the original agg output to agg output expr. // for comparison predicate, it is always the child(1), since we ensure the window agg slot is in child(0) @@ -269,9 +352,9 @@ public class AggScalarSubQueryToWindowFunction extends DefaultPlanRewriter(ImmutableList.of(windowFunctionAlias), newFilter); - LogicalFilter windowFilter = new LogicalFilter<>(ImmutableSet.of(windowFilterConjunct), newWindow); + LogicalFilter newFilter = (LogicalFilter) filter.withChildren(apply.left()); + LogicalWindow newWindow = new LogicalWindow<>(ImmutableList.of(windowFunctionAlias), newFilter); + LogicalFilter windowFilter = new LogicalFilter<>(ImmutableSet.of(windowFilterConjunct), newWindow); return windowFilter; } @@ -281,24 +364,6 @@ public class AggScalarSubQueryToWindowFunction extends DefaultPlanRewriter> { - public static final PlanCollector INSTANCE = new PlanCollector(); - - public List collect(LogicalPlan plan) { - List buffer = Lists.newArrayList(); - plan.accept(this, buffer); - return buffer; - } - - @Override - public Void visit(Plan plan, List buffer) { - Preconditions.checkArgument(plan instanceof LogicalPlan); - buffer.add(((LogicalPlan) plan)); - plan.children().forEach(child -> child.accept(this, buffer)); - return null; - } - } - private static class ExpressionIdenticalChecker extends DefaultExpressionVisitor { public static final ExpressionIdenticalChecker INSTANCE = new ExpressionIdenticalChecker(); @@ -322,64 +387,24 @@ public class AggScalarSubQueryToWindowFunction extends DefaultPlanRewriter, - Function>> { - public static final MapReplacer INSTANCE = new MapReplacer(); - - public Expression replace(Expression e, Map, - Function> context) { - return e.accept(this, context); - } - - @Override - public Expression visit(Expression e, Map, - Function> context) { - Expression replaced = e; - for (Class c : context.keySet()) { - if (c.isInstance(e)) { - replaced = context.get(c).apply(e); - break; - } - } - return super.visit(replaced, context); + public Boolean visitComparisonPredicate(ComparisonPredicate cp, Expression other) { + return cp.equals(other) || cp.commute().equals(other); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Avg.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Avg.java index 2124e15b24..48dff01884 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Avg.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Avg.java @@ -23,6 +23,7 @@ import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.functions.ComputePrecision; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.window.SupportWindowAnalytic; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.BigIntType; @@ -43,7 +44,7 @@ import java.util.List; * AggregateFunction 'avg'. This class is generated by GenerateFunction. */ public class Avg extends NullableAggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, ComputePrecision { + implements UnaryExpression, ExplicitlyCastableSignature, ComputePrecision, SupportWindowAnalytic { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(DoubleType.INSTANCE).args(TinyIntType.INSTANCE), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Count.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Count.java index 434f43baaf..6212210816 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Count.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Count.java @@ -23,6 +23,7 @@ import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.window.SupportWindowAnalytic; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.BigIntType; import org.apache.doris.nereids.types.DataType; @@ -35,7 +36,7 @@ import java.util.List; /** count agg function. */ public class Count extends AggregateFunction - implements ExplicitlyCastableSignature, AlwaysNotNullable { + implements ExplicitlyCastableSignature, AlwaysNotNullable, SupportWindowAnalytic { public static final List SIGNATURES = ImmutableList.of( // count(*) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Max.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Max.java index 5a6ff23b41..02ac778eea 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Max.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Max.java @@ -22,6 +22,7 @@ import org.apache.doris.catalog.Type; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.functions.CustomSignature; +import org.apache.doris.nereids.trees.expressions.functions.window.SupportWindowAnalytic; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.DataType; @@ -32,7 +33,8 @@ import com.google.common.collect.ImmutableList; import java.util.List; /** max agg function. */ -public class Max extends NullableAggregateFunction implements UnaryExpression, CustomSignature { +public class Max extends NullableAggregateFunction + implements UnaryExpression, CustomSignature, SupportWindowAnalytic { public Max(Expression child) { this(false, false, child); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Min.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Min.java index 7103e16519..ef3fd8db32 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Min.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Min.java @@ -22,6 +22,7 @@ import org.apache.doris.catalog.Type; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.functions.CustomSignature; +import org.apache.doris.nereids.trees.expressions.functions.window.SupportWindowAnalytic; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.DataType; @@ -32,7 +33,8 @@ import com.google.common.collect.ImmutableList; import java.util.List; /** min agg function. */ -public class Min extends NullableAggregateFunction implements UnaryExpression, CustomSignature { +public class Min extends NullableAggregateFunction + implements UnaryExpression, CustomSignature, SupportWindowAnalytic { public Min(Expression child) { this(false, false, child); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Sum.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Sum.java index 93788bc016..e91875902c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Sum.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/agg/Sum.java @@ -22,6 +22,7 @@ import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.functions.ComputePrecisionForSum; import org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature; +import org.apache.doris.nereids.trees.expressions.functions.window.SupportWindowAnalytic; import org.apache.doris.nereids.trees.expressions.shape.UnaryExpression; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.BigIntType; @@ -43,7 +44,7 @@ import java.util.List; * AggregateFunction 'sum'. This class is generated by GenerateFunction. */ public class Sum extends NullableAggregateFunction - implements UnaryExpression, ExplicitlyCastableSignature, ComputePrecisionForSum { + implements UnaryExpression, ExplicitlyCastableSignature, ComputePrecisionForSum, SupportWindowAnalytic { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BigIntType.INSTANCE).args(TinyIntType.INSTANCE), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/SupportWindowAnalytic.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/SupportWindowAnalytic.java new file mode 100644 index 0000000000..af43466ca0 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/SupportWindowAnalytic.java @@ -0,0 +1,24 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.expressions.functions.window; + +/** + * function that support window analytic + */ +public interface SupportWindowAnalytic { +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/WindowFunction.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/WindowFunction.java index 2f825c17bd..c30372eec2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/WindowFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/window/WindowFunction.java @@ -26,7 +26,7 @@ import java.util.Objects; /** * Window functions, as known as analytic functions. */ -public abstract class WindowFunction extends BoundFunction { +public abstract class WindowFunction extends BoundFunction implements SupportWindowAnalytic { public WindowFunction(String name, Expression... arguments) { super(name, arguments); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/logical/AggScalarSubQueryToWindowFunctionTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/logical/AggScalarSubQueryToWindowFunctionTest.java index d63385f656..dff9f63735 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/logical/AggScalarSubQueryToWindowFunctionTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/logical/AggScalarSubQueryToWindowFunctionTest.java @@ -98,6 +98,7 @@ public class AggScalarSubQueryToWindowFunctionTest extends TPCHTestBase implemen @Test public void testNotMatchTheRule() { String[] testCases = { + // not correlated "select sum(l_extendedprice) / 7.0 as avg_yearly\n" + " from lineitem, part\n" + " where p_partkey = l_partkey\n" @@ -106,6 +107,7 @@ public class AggScalarSubQueryToWindowFunctionTest extends TPCHTestBase implemen + " and l_quantity < (\n" + " select 0.2 * avg(l_quantity)\n" + " from lineitem);", + // no standalone correlated table in outer scope "select sum(l_extendedprice) / 7.0 as avg_yearly\n" + " from lineitem, part\n" + " where p_partkey = l_partkey\n" @@ -115,6 +117,7 @@ public class AggScalarSubQueryToWindowFunctionTest extends TPCHTestBase implemen + " select 0.2 * avg(l_quantity)\n" + " from lineitem, part\n" + " where l_partkey = p_partkey);", + // inner scope table not appear in outer scope "select sum(l_extendedprice) / 7.0 as avg_yearly\n" + " from lineitem, part\n" + " where p_partkey = l_partkey\n" @@ -124,6 +127,7 @@ public class AggScalarSubQueryToWindowFunctionTest extends TPCHTestBase implemen + " select 0.2 * avg(l_quantity)\n" + " from lineitem, partsupp\n" + " where l_partkey = p_partkey);", + // inner filter not a subset of outer filter "select sum(l_extendedprice) / 7.0 as avg_yearly\n" + " from lineitem, part\n" + " where\n" @@ -135,6 +139,7 @@ public class AggScalarSubQueryToWindowFunctionTest extends TPCHTestBase implemen + " from lineitem\n" + " where l_partkey = p_partkey\n" + " and p_brand = 'Brand#24');", + // inner filter not a subset of outer filter "select sum(l_extendedprice) / 7.0 as avg_yearly\n" + " from lineitem, part\n" + " where\n" @@ -145,7 +150,186 @@ public class AggScalarSubQueryToWindowFunctionTest extends TPCHTestBase implemen + " select 0.2 * avg(l_quantity)\n" + " from lineitem\n" + " where l_partkey = p_partkey\n" - + " and l_partkey = 10);" + + " and l_partkey = 10);", + // inner has a mapping project + "select\n" + + " sum(l_extendedprice) / 7.0 as avg_yearly\n" + + "from\n" + + " lineitem,\n" + + " part\n" + + "where\n" + + " p_partkey = l_partkey\n" + + " and p_brand = 'Brand#23'\n" + + " and p_container = 'MED BOX'\n" + + " and l_quantity < (\n" + + " select\n" + + " 0.2 * avg(l_quantity)\n" + + " from\n" + + " (select l_partkey, l_quantity * 0.2 as l_quantity from lineitem) b\n" + + " where\n" + + " l_partkey = p_partkey\n" + + " );", + // inner has an unexpected filter + "select\n" + + " sum(l_extendedprice) / 7.0 as avg_yearly\n" + + "from\n" + + " lineitem,\n" + + " part\n" + + "where\n" + + " p_partkey = l_partkey\n" + + " and p_brand = 'Brand#23'\n" + + " and p_container = 'MED BOX'\n" + + " and l_quantity < (\n" + + " select\n" + + " 0.2 * avg(l_quantity)\n" + + " from\n" + + " (select l_partkey, l_quantity from lineitem where l_quantity > 5) b\n" + + " where\n" + + " l_partkey = p_partkey\n" + + " );", + // outer has a mapping project + "select\n" + + " sum(l_extendedprice) / 7.0 as avg_yearly\n" + + "from\n" + + " (select l_extendedprice, l_partkey, l_quantity * 0.2 as l_quantity from lineitem) b,\n" + + " part\n" + + "where\n" + + " p_partkey = l_partkey\n" + + " and p_brand = 'Brand#23'\n" + + " and p_container = 'MED BOX'\n" + + " and l_quantity < (\n" + + " select\n" + + " 0.2 * avg(l_quantity)\n" + + " from\n" + + " lineitem\n" + + " where\n" + + " l_partkey = p_partkey\n" + + " );", + // outer has an unexpected filter + "select\n" + + " sum(l_extendedprice) / 7.0 as avg_yearly\n" + + "from\n" + + " (select l_extendedprice, l_partkey, l_quantity from lineitem where l_quantity > 5) b,\n" + + " part\n" + + "where\n" + + " p_partkey = l_partkey\n" + + " and p_brand = 'Brand#23'\n" + + " and p_container = 'MED BOX'\n" + + " and l_quantity < (\n" + + " select\n" + + " 0.2 * avg(l_quantity)\n" + + " from\n" + + " lineitem\n" + + " where\n" + + " l_partkey = p_partkey\n" + + " );", + // outer has additional table + "select\n" + + " sum(l_extendedprice) / 7.0 as avg_yearly\n" + + "from\n" + + " orders,\n" + + " lineitem,\n" + + " part\n" + + "where\n" + + " p_partkey = l_partkey\n" + + " and O_SHIPPRIORITY = 5\n" + + " and O_ORDERKEY = L_ORDERKEY\n" + + " and p_brand = 'Brand#23'\n" + + " and p_container = 'MED BOX'\n" + + " and l_quantity < (\n" + + " select\n" + + " 0.2 * avg(l_quantity)\n" + + " from\n" + + " lineitem\n" + + " where\n" + + " l_partkey = p_partkey\n" + + " );", + // outer has same table + "select\n" + + " sum(l1.l_extendedprice) / 7.0 as avg_yearly\n" + + "from\n" + + " lineitem l2,\n" + + " lineitem l1,\n" + + " part\n" + + "where\n" + + " p_partkey = l1.l_partkey\n" + + " and l2.l_partkey = 5\n" + + " and l2.l_partkey = l1.l_partkey\n" + + " and p_brand = 'Brand#23'\n" + + " and p_container = 'MED BOX'\n" + + " and l1.l_quantity < (\n" + + " select\n" + + " 0.2 * avg(l_quantity)\n" + + " from\n" + + " lineitem\n" + + " where\n" + + " l_partkey = p_partkey\n" + + " );", + // outer and inner with different join condition + "select\n" + + " s_acctbal,\n" + + " s_name,\n" + + " n_name,\n" + + " p_partkey,\n" + + " p_mfgr,\n" + + " s_address,\n" + + " s_phone,\n" + + " s_comment\n" + + "from\n" + + " part,\n" + + " supplier,\n" + + " partsupp,\n" + + " nation join\n" + + " region on n_regionkey = r_regionkey\n" + + "where\n" + + " p_partkey = ps_partkey\n" + + " and s_suppkey = ps_suppkey\n" + + " and p_size = 15\n" + + " and p_type like '%BRASS'\n" + + " and s_nationkey = n_nationkey\n" + + " and r_name = 'EUROPE'\n" + + " and ps_supplycost = (\n" + + " select\n" + + " min(ps_supplycost)\n" + + " from\n" + + " partsupp,\n" + + " supplier,\n" + + " nation join\n" + + " region on n_regionkey = r_regionkey + 1\n" + + " where\n" + + " p_partkey = ps_partkey\n" + + " and s_suppkey = ps_suppkey\n" + + " and s_nationkey = n_nationkey\n" + + " and r_name = 'EUROPE'\n" + + " )\n" + + "order by\n" + + " s_acctbal desc,\n" + + " n_name,\n" + + " s_name,\n" + + " p_partkey\n" + + "limit 100;", + // outer and inner has same table with different join condition + "select\n" + + " sum(l1.l_extendedprice) / 7.0 as avg_yearly\n" + + "from\n" + + " lineitem l1,\n" + + " lineitem l2,\n" + + " part\n" + + "where\n" + + " l2.l_quantity + 1 = l1.l_quantity\n" + + " and p_partkey = l1.l_partkey\n" + + " and p_brand = 'Brand#23'\n" + + " and p_container = 'MED BOX'\n" + + " and l1.l_quantity < (\n" + + " select\n" + + " 0.2 * avg(l1.l_quantity)\n" + + " from\n" + + " lineitem l1,\n" + + " lineitem l2\n" + + " where\n" + + " l1.l_quantity = l2.l_quantity + 1\n" + + " and l2.l_partkey = p_partkey\n" + + " );" }; // notice: case 4 and 5 can apply the rule, but we support it later. for (String s : testCases) { diff --git a/regression-test/data/nereids_tpch_p0/tpch/q1.out b/regression-test/data/nereids_tpch_p0/tpch/q1.out index 7671e89987..71385e310e 100644 --- a/regression-test/data/nereids_tpch_p0/tpch/q1.out +++ b/regression-test/data/nereids_tpch_p0/tpch/q1.out @@ -1,13 +1,13 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -A F 3774200.000000000 5320753880.690000000 5054096266.682800000 5256751331.449234000 25.537587117 36002.123829014 0.050144597 147790 -N F 95257.000000000 133737795.840000000 127132372.651200000 132286291.229445000 25.300664011 35521.326916335 0.049394422 3765 -N O 7459297.000000000 10512270008.900000000 9986238338.384700000 10385578376.585467000 25.545537671 36000.924688014 0.050095959 292000 -R F 3785523.000000000 5337950526.470000000 5071818532.942000000 5274405503.049367000 25.525943857 35994.029214031 0.049989279 148301 +A F 3774200.00 5320753880.69 5054096266.6828 5256751331.449234 25.5375 36002.1238 0.0501 147790 +N F 95257.00 133737795.84 127132372.6512 132286291.229445 25.3006 35521.3269 0.0493 3765 +N O 7459297.00 10512270008.90 9986238338.3847 10385578376.585467 25.5455 36000.9246 0.0500 292000 +R F 3785523.00 5337950526.47 5071818532.9420 5274405503.049367 25.5259 35994.0292 0.0499 148301 -- !select -- -A F 3774200.000000000 5320753880.690000000 5054096266.682800000 5256751331.449234000 25.537587117 36002.123829014 0.050144597 147790 -N F 95257.000000000 133737795.840000000 127132372.651200000 132286291.229445000 25.300664011 35521.326916335 0.049394422 3765 -N O 7459297.000000000 10512270008.900000000 9986238338.384700000 10385578376.585467000 25.545537671 36000.924688014 0.050095959 292000 -R F 3785523.000000000 5337950526.470000000 5071818532.942000000 5274405503.049367000 25.525943857 35994.029214031 0.049989279 148301 +A F 3774200.00 5320753880.69 5054096266.6828 5256751331.449234 25.5375 36002.1238 0.0501 147790 +N F 95257.00 133737795.84 127132372.6512 132286291.229445 25.3006 35521.3269 0.0493 3765 +N O 7459297.00 10512270008.90 9986238338.3847 10385578376.585467 25.5455 36000.9246 0.0500 292000 +R F 3785523.00 5337950526.47 5071818532.9420 5274405503.049367 25.5259 35994.0292 0.0499 148301 diff --git a/regression-test/data/nereids_tpch_p0/tpch/q10.out b/regression-test/data/nereids_tpch_p0/tpch/q10.out index ffb56aff29..e9c182ff3c 100644 --- a/regression-test/data/nereids_tpch_p0/tpch/q10.out +++ b/regression-test/data/nereids_tpch_p0/tpch/q10.out @@ -1,45 +1,45 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -8242 Customer#000008242 622786.729700000 6322.090000000 ETHIOPIA P2n4nJhy,UqSo2s43YfSvYJDZ6lk 15-792-676-1184 slyly regular packages haggle carefully ironic ideas. courts are furiously. furiously unusual theodolites cajole. i -7714 Customer#000007714 557400.305300000 9799.980000000 IRAN SnnIGB,SkmnWpX3 20-922-418-6024 arhorses according to the blithely express re -11032 Customer#000011032 512500.964100000 8496.930000000 UNITED KINGDOM WIKHC7K3Cn7156iNOyfVG3cZ7YqkgsR,Ly 33-102-772-3533 posits-- furiously ironic accounts are again -2455 Customer#000002455 481592.405300000 2070.990000000 GERMANY RVn1ZSRtLqPlJLIZxvpmsbgC02 17-946-225-9977 al asymptotes. finally ironic accounts cajole furiously. permanently unusual theodolites aro -12106 Customer#000012106 479414.213300000 5342.110000000 UNITED STATES wth3twOmu6vy 34-905-346-4472 ly after the blithely regular foxes. accounts haggle carefully alongside of the blithely even ideas. -8530 Customer#000008530 457855.946700000 9734.950000000 MOROCCO GMQyte94oDM7eD7exnkj 4hH9yq3 25-736-932-5850 slyly asymptotes. quickly final deposits in -13984 Customer#000013984 446316.510400000 3482.280000000 IRAN qZXwuapCHvxbX 20-981-264-2952 y unusual courts could wake furiously -1966 Customer#000001966 444059.038200000 1937.720000000 ALGERIA jPv1 UHra5JLALR5Isci5u0636RoAu7t vH 10-973-269-8886 the blithely even accounts. final deposits cajole around the blithely final packages. -11026 Customer#000011026 417913.414200000 7738.760000000 ALGERIA XorIktoJOAEJkpNNMx 10-184-163-4632 ly even dolphins eat along the blithely even instructions. express attainments cajole slyly. busy dolphins in -8501 Customer#000008501 412797.510000000 6906.700000000 ARGENTINA 776af4rOa mZ66hczs 11-317-552-5840 y final deposits after the fluffily even accounts are slyly final, regular -1565 Customer#000001565 412506.006200000 1820.030000000 BRAZIL EWQO5Ck,nMuHVQimqL8dLrixRP6QKveXcz9QgorW 12-402-178-2007 ously regular accounts wake slyly ironic idea -14398 Customer#000014398 408575.360000000 -602.240000000 UNITED STATES GWRCgIPHajtU21vICVvbJJerFu2cUk 34-814-111-5424 s. blithely even accounts cajole blithely. even foxes doubt-- -1465 Customer#000001465 405055.345700000 9365.930000000 INDIA tDRaTC7UgFbBX7VF6cVXYQA0 18-807-487-1074 s lose blithely ironic, regular packages. regular, final foxes haggle c -12595 Customer#000012595 401402.239100000 -6.920000000 INDIA LmeaX5cR,w9NqKugl yRm98 18-186-132-3352 o the busy accounts. blithely special gifts maintain a -961 Customer#000000961 401198.173700000 6963.680000000 JAPAN 5,81YDLFuRR47KKzv8GXdmi3zyP37PlPn 22-989-463-6089 e final requests: busily final accounts believe a -14299 Customer#000014299 400968.375100000 6595.970000000 RUSSIA 7lFczTya0iM1bhEWT 32-156-618-1224 carefully regular requests. quickly ironic accounts against the ru -623 Customer#000000623 399883.425700000 7887.600000000 INDONESIA HXiFb9oWlgqZXrJPUCEJ6zZIPxAM4m6 19-113-202-7085 requests. dolphins above the busily regular dependencies cajole after -9151 Customer#000009151 396562.029500000 5691.950000000 IRAQ 7gIdRdaxB91EVdyx8DyPjShpMD 21-834-147-4906 ajole fluffily. furiously regular accounts are special, silent account -14819 Customer#000014819 396271.103600000 7308.390000000 FRANCE w8StIbymUXmLCcUag6sx6LUIp8E3pA,Ux 16-769-398-7926 ss, final asymptotes use furiously slyly ironic dependencies. special, express dugouts according to the dep -13478 Customer#000013478 395513.135800000 -778.110000000 KENYA 9VIsvIeZrJpC6OOdYheMC2vdtq8Ai0Rt 24-983-202-8240 r theodolites. slyly unusual pinto beans sleep fluffily against the asymptotes. quickly r +8242 Customer#000008242 622786.7297 6322.09 ETHIOPIA P2n4nJhy,UqSo2s43YfSvYJDZ6lk 15-792-676-1184 slyly regular packages haggle carefully ironic ideas. courts are furiously. furiously unusual theodolites cajole. i +7714 Customer#000007714 557400.3053 9799.98 IRAN SnnIGB,SkmnWpX3 20-922-418-6024 arhorses according to the blithely express re +11032 Customer#000011032 512500.9641 8496.93 UNITED KINGDOM WIKHC7K3Cn7156iNOyfVG3cZ7YqkgsR,Ly 33-102-772-3533 posits-- furiously ironic accounts are again +2455 Customer#000002455 481592.4053 2070.99 GERMANY RVn1ZSRtLqPlJLIZxvpmsbgC02 17-946-225-9977 al asymptotes. finally ironic accounts cajole furiously. permanently unusual theodolites aro +12106 Customer#000012106 479414.2133 5342.11 UNITED STATES wth3twOmu6vy 34-905-346-4472 ly after the blithely regular foxes. accounts haggle carefully alongside of the blithely even ideas. +8530 Customer#000008530 457855.9467 9734.95 MOROCCO GMQyte94oDM7eD7exnkj 4hH9yq3 25-736-932-5850 slyly asymptotes. quickly final deposits in +13984 Customer#000013984 446316.5104 3482.28 IRAN qZXwuapCHvxbX 20-981-264-2952 y unusual courts could wake furiously +1966 Customer#000001966 444059.0382 1937.72 ALGERIA jPv1 UHra5JLALR5Isci5u0636RoAu7t vH 10-973-269-8886 the blithely even accounts. final deposits cajole around the blithely final packages. +11026 Customer#000011026 417913.4142 7738.76 ALGERIA XorIktoJOAEJkpNNMx 10-184-163-4632 ly even dolphins eat along the blithely even instructions. express attainments cajole slyly. busy dolphins in +8501 Customer#000008501 412797.5100 6906.70 ARGENTINA 776af4rOa mZ66hczs 11-317-552-5840 y final deposits after the fluffily even accounts are slyly final, regular +1565 Customer#000001565 412506.0062 1820.03 BRAZIL EWQO5Ck,nMuHVQimqL8dLrixRP6QKveXcz9QgorW 12-402-178-2007 ously regular accounts wake slyly ironic idea +14398 Customer#000014398 408575.3600 -602.24 UNITED STATES GWRCgIPHajtU21vICVvbJJerFu2cUk 34-814-111-5424 s. blithely even accounts cajole blithely. even foxes doubt-- +1465 Customer#000001465 405055.3457 9365.93 INDIA tDRaTC7UgFbBX7VF6cVXYQA0 18-807-487-1074 s lose blithely ironic, regular packages. regular, final foxes haggle c +12595 Customer#000012595 401402.2391 -6.92 INDIA LmeaX5cR,w9NqKugl yRm98 18-186-132-3352 o the busy accounts. blithely special gifts maintain a +961 Customer#000000961 401198.1737 6963.68 JAPAN 5,81YDLFuRR47KKzv8GXdmi3zyP37PlPn 22-989-463-6089 e final requests: busily final accounts believe a +14299 Customer#000014299 400968.3751 6595.97 RUSSIA 7lFczTya0iM1bhEWT 32-156-618-1224 carefully regular requests. quickly ironic accounts against the ru +623 Customer#000000623 399883.4257 7887.60 INDONESIA HXiFb9oWlgqZXrJPUCEJ6zZIPxAM4m6 19-113-202-7085 requests. dolphins above the busily regular dependencies cajole after +9151 Customer#000009151 396562.0295 5691.95 IRAQ 7gIdRdaxB91EVdyx8DyPjShpMD 21-834-147-4906 ajole fluffily. furiously regular accounts are special, silent account +14819 Customer#000014819 396271.1036 7308.39 FRANCE w8StIbymUXmLCcUag6sx6LUIp8E3pA,Ux 16-769-398-7926 ss, final asymptotes use furiously slyly ironic dependencies. special, express dugouts according to the dep +13478 Customer#000013478 395513.1358 -778.11 KENYA 9VIsvIeZrJpC6OOdYheMC2vdtq8Ai0Rt 24-983-202-8240 r theodolites. slyly unusual pinto beans sleep fluffily against the asymptotes. quickly r -- !select -- -8242 Customer#000008242 622786.729700000 6322.090000000 ETHIOPIA P2n4nJhy,UqSo2s43YfSvYJDZ6lk 15-792-676-1184 slyly regular packages haggle carefully ironic ideas. courts are furiously. furiously unusual theodolites cajole. i -7714 Customer#000007714 557400.305300000 9799.980000000 IRAN SnnIGB,SkmnWpX3 20-922-418-6024 arhorses according to the blithely express re -11032 Customer#000011032 512500.964100000 8496.930000000 UNITED KINGDOM WIKHC7K3Cn7156iNOyfVG3cZ7YqkgsR,Ly 33-102-772-3533 posits-- furiously ironic accounts are again -2455 Customer#000002455 481592.405300000 2070.990000000 GERMANY RVn1ZSRtLqPlJLIZxvpmsbgC02 17-946-225-9977 al asymptotes. finally ironic accounts cajole furiously. permanently unusual theodolites aro -12106 Customer#000012106 479414.213300000 5342.110000000 UNITED STATES wth3twOmu6vy 34-905-346-4472 ly after the blithely regular foxes. accounts haggle carefully alongside of the blithely even ideas. -8530 Customer#000008530 457855.946700000 9734.950000000 MOROCCO GMQyte94oDM7eD7exnkj 4hH9yq3 25-736-932-5850 slyly asymptotes. quickly final deposits in -13984 Customer#000013984 446316.510400000 3482.280000000 IRAN qZXwuapCHvxbX 20-981-264-2952 y unusual courts could wake furiously -1966 Customer#000001966 444059.038200000 1937.720000000 ALGERIA jPv1 UHra5JLALR5Isci5u0636RoAu7t vH 10-973-269-8886 the blithely even accounts. final deposits cajole around the blithely final packages. -11026 Customer#000011026 417913.414200000 7738.760000000 ALGERIA XorIktoJOAEJkpNNMx 10-184-163-4632 ly even dolphins eat along the blithely even instructions. express attainments cajole slyly. busy dolphins in -8501 Customer#000008501 412797.510000000 6906.700000000 ARGENTINA 776af4rOa mZ66hczs 11-317-552-5840 y final deposits after the fluffily even accounts are slyly final, regular -1565 Customer#000001565 412506.006200000 1820.030000000 BRAZIL EWQO5Ck,nMuHVQimqL8dLrixRP6QKveXcz9QgorW 12-402-178-2007 ously regular accounts wake slyly ironic idea -14398 Customer#000014398 408575.360000000 -602.240000000 UNITED STATES GWRCgIPHajtU21vICVvbJJerFu2cUk 34-814-111-5424 s. blithely even accounts cajole blithely. even foxes doubt-- -1465 Customer#000001465 405055.345700000 9365.930000000 INDIA tDRaTC7UgFbBX7VF6cVXYQA0 18-807-487-1074 s lose blithely ironic, regular packages. regular, final foxes haggle c -12595 Customer#000012595 401402.239100000 -6.920000000 INDIA LmeaX5cR,w9NqKugl yRm98 18-186-132-3352 o the busy accounts. blithely special gifts maintain a -961 Customer#000000961 401198.173700000 6963.680000000 JAPAN 5,81YDLFuRR47KKzv8GXdmi3zyP37PlPn 22-989-463-6089 e final requests: busily final accounts believe a -14299 Customer#000014299 400968.375100000 6595.970000000 RUSSIA 7lFczTya0iM1bhEWT 32-156-618-1224 carefully regular requests. quickly ironic accounts against the ru -623 Customer#000000623 399883.425700000 7887.600000000 INDONESIA HXiFb9oWlgqZXrJPUCEJ6zZIPxAM4m6 19-113-202-7085 requests. dolphins above the busily regular dependencies cajole after -9151 Customer#000009151 396562.029500000 5691.950000000 IRAQ 7gIdRdaxB91EVdyx8DyPjShpMD 21-834-147-4906 ajole fluffily. furiously regular accounts are special, silent account -14819 Customer#000014819 396271.103600000 7308.390000000 FRANCE w8StIbymUXmLCcUag6sx6LUIp8E3pA,Ux 16-769-398-7926 ss, final asymptotes use furiously slyly ironic dependencies. special, express dugouts according to the dep -13478 Customer#000013478 395513.135800000 -778.110000000 KENYA 9VIsvIeZrJpC6OOdYheMC2vdtq8Ai0Rt 24-983-202-8240 r theodolites. slyly unusual pinto beans sleep fluffily against the asymptotes. quickly r +8242 Customer#000008242 622786.7297 6322.09 ETHIOPIA P2n4nJhy,UqSo2s43YfSvYJDZ6lk 15-792-676-1184 slyly regular packages haggle carefully ironic ideas. courts are furiously. furiously unusual theodolites cajole. i +7714 Customer#000007714 557400.3053 9799.98 IRAN SnnIGB,SkmnWpX3 20-922-418-6024 arhorses according to the blithely express re +11032 Customer#000011032 512500.9641 8496.93 UNITED KINGDOM WIKHC7K3Cn7156iNOyfVG3cZ7YqkgsR,Ly 33-102-772-3533 posits-- furiously ironic accounts are again +2455 Customer#000002455 481592.4053 2070.99 GERMANY RVn1ZSRtLqPlJLIZxvpmsbgC02 17-946-225-9977 al asymptotes. finally ironic accounts cajole furiously. permanently unusual theodolites aro +12106 Customer#000012106 479414.2133 5342.11 UNITED STATES wth3twOmu6vy 34-905-346-4472 ly after the blithely regular foxes. accounts haggle carefully alongside of the blithely even ideas. +8530 Customer#000008530 457855.9467 9734.95 MOROCCO GMQyte94oDM7eD7exnkj 4hH9yq3 25-736-932-5850 slyly asymptotes. quickly final deposits in +13984 Customer#000013984 446316.5104 3482.28 IRAN qZXwuapCHvxbX 20-981-264-2952 y unusual courts could wake furiously +1966 Customer#000001966 444059.0382 1937.72 ALGERIA jPv1 UHra5JLALR5Isci5u0636RoAu7t vH 10-973-269-8886 the blithely even accounts. final deposits cajole around the blithely final packages. +11026 Customer#000011026 417913.4142 7738.76 ALGERIA XorIktoJOAEJkpNNMx 10-184-163-4632 ly even dolphins eat along the blithely even instructions. express attainments cajole slyly. busy dolphins in +8501 Customer#000008501 412797.5100 6906.70 ARGENTINA 776af4rOa mZ66hczs 11-317-552-5840 y final deposits after the fluffily even accounts are slyly final, regular +1565 Customer#000001565 412506.0062 1820.03 BRAZIL EWQO5Ck,nMuHVQimqL8dLrixRP6QKveXcz9QgorW 12-402-178-2007 ously regular accounts wake slyly ironic idea +14398 Customer#000014398 408575.3600 -602.24 UNITED STATES GWRCgIPHajtU21vICVvbJJerFu2cUk 34-814-111-5424 s. blithely even accounts cajole blithely. even foxes doubt-- +1465 Customer#000001465 405055.3457 9365.93 INDIA tDRaTC7UgFbBX7VF6cVXYQA0 18-807-487-1074 s lose blithely ironic, regular packages. regular, final foxes haggle c +12595 Customer#000012595 401402.2391 -6.92 INDIA LmeaX5cR,w9NqKugl yRm98 18-186-132-3352 o the busy accounts. blithely special gifts maintain a +961 Customer#000000961 401198.1737 6963.68 JAPAN 5,81YDLFuRR47KKzv8GXdmi3zyP37PlPn 22-989-463-6089 e final requests: busily final accounts believe a +14299 Customer#000014299 400968.3751 6595.97 RUSSIA 7lFczTya0iM1bhEWT 32-156-618-1224 carefully regular requests. quickly ironic accounts against the ru +623 Customer#000000623 399883.4257 7887.60 INDONESIA HXiFb9oWlgqZXrJPUCEJ6zZIPxAM4m6 19-113-202-7085 requests. dolphins above the busily regular dependencies cajole after +9151 Customer#000009151 396562.0295 5691.95 IRAQ 7gIdRdaxB91EVdyx8DyPjShpMD 21-834-147-4906 ajole fluffily. furiously regular accounts are special, silent account +14819 Customer#000014819 396271.1036 7308.39 FRANCE w8StIbymUXmLCcUag6sx6LUIp8E3pA,Ux 16-769-398-7926 ss, final asymptotes use furiously slyly ironic dependencies. special, express dugouts according to the dep +13478 Customer#000013478 395513.1358 -778.11 KENYA 9VIsvIeZrJpC6OOdYheMC2vdtq8Ai0Rt 24-983-202-8240 r theodolites. slyly unusual pinto beans sleep fluffily against the asymptotes. quickly r diff --git a/regression-test/data/nereids_tpch_p0/tpch/q11.out b/regression-test/data/nereids_tpch_p0/tpch/q11.out index 7882e65242..16219392ad 100644 --- a/regression-test/data/nereids_tpch_p0/tpch/q11.out +++ b/regression-test/data/nereids_tpch_p0/tpch/q11.out @@ -1,6217 +1,6217 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -12098 16227681.210000000 -5134 15709338.520000000 -13334 15023662.410000000 -17052 14351644.200000000 -3452 14070870.140000000 -12552 13332469.180000000 -1084 13170428.290000000 -5797 13038622.720000000 -12633 12892561.610000000 -403 12856217.340000000 -1833 12024581.720000000 -2084 11502875.360000000 -17349 11354213.050000000 -18427 11282385.240000000 -2860 11262529.950000000 -17852 10934711.930000000 -9871 10889253.680000000 -12231 10841131.390000000 -6366 10759786.810000000 -12146 10257362.660000000 -5043 10226395.880000000 -12969 10125777.930000000 -1504 10004397.080000000 -14327 9981697.080000000 -134 9965150.660000000 -6860 9805871.260000000 -10624 9776138.400000000 -15819 9775705.310000000 -3293 9674928.120000000 -19865 9653766.830000000 -8870 9648981.870000000 -15778 9636332.820000000 -12360 9635023.920000000 -14389 9475588.340000000 -3257 9451029.240000000 -9476 9435207.280000000 -19629 9391236.400000000 -7179 9386222.250000000 -15723 9383900.800000000 -4054 9313810.020000000 -2380 9307751.220000000 -19084 9302916.800000000 -4703 9280804.800000000 -18791 9267017.970000000 -19994 9235972.920000000 -9149 9121803.900000000 -15118 9120819.500000000 -6116 9079369.200000000 -7052 9077468.920000000 -14147 9069193.780000000 -7305 9035228.530000000 -9130 9024379.250000000 -16698 8991337.950000000 -1553 8977226.100000000 -16777 8961355.620000000 -1402 8953779.120000000 -18963 8934063.400000000 -8358 8930611.480000000 -17547 8860117.000000000 -5128 8844222.750000000 -17063 8840649.600000000 -15490 8833581.400000000 -14761 8817240.560000000 -19601 8791341.020000000 -16160 8740262.760000000 -13597 8702669.820000000 -13653 8693170.160000000 -16383 8691505.920000000 -325 8667741.280000000 -8879 8667584.380000000 -10564 8667098.220000000 -17429 8661827.900000000 -17403 8643350.300000000 -18294 8616583.430000000 -4181 8592684.660000000 -13008 8567480.640000000 -13211 8537000.010000000 -1884 8532644.340000000 -11101 8530945.320000000 -11562 8528028.570000000 -15878 8523591.840000000 -834 8522135.270000000 -2423 8517902.850000000 -15383 8513433.110000000 -18119 8507611.800000000 -7389 8506099.200000000 -5016 8489784.150000000 -17473 8444766.240000000 -6669 8428618.460000000 -384 8418472.270000000 -12052 8411519.280000000 -17562 8409022.830000000 -8128 8379149.470000000 -13813 8374830.840000000 -12800 8318626.780000000 -10887 8315019.360000000 -1644 8285453.080000000 -16638 8274568.000000000 -1394 8255140.600000000 -7219 8254985.300000000 -13358 8253829.800000000 -5562 8252365.160000000 -14861 8242296.150000000 -15416 8196621.530000000 -1963 8192206.610000000 -2841 8148678.450000000 -6635 8122918.280000000 -3609 8099812.050000000 -6372 8093695.060000000 -5231 8091704.180000000 -8755 8085017.030000000 -4071 8083755.180000000 -4977 8058501.720000000 -11873 8057540.300000000 -12127 8051142.940000000 -2401 8049525.940000000 -15964 8037547.750000000 -10129 8030855.040000000 -7605 8028622.420000000 -9327 8022291.300000000 -11814 7983589.590000000 -4646 7981660.600000000 -6032 7981511.280000000 -1076 7977074.750000000 -4043 7971929.900000000 -8932 7967222.190000000 -13705 7953344.520000000 -16855 7923819.000000000 -3234 7920022.000000000 -17752 7901362.770000000 -2097 7892993.170000000 -18599 7890774.400000000 -19596 7874564.590000000 -11392 7861172.480000000 -18121 7857581.600000000 -17452 7838465.360000000 -6076 7821316.800000000 -15134 7804760.250000000 -8641 7802917.150000000 -2134 7800185.430000000 -16352 7797112.000000000 -19312 7775952.240000000 -2895 7759327.230000000 -12860 7758135.210000000 -153 7755681.280000000 -15089 7735438.260000000 -14797 7725353.160000000 -15946 7722773.880000000 -10919 7722425.360000000 -9867 7721597.780000000 -11881 7713136.420000000 -16552 7708518.150000000 -6925 7703999.680000000 -12147 7703826.980000000 -8923 7702690.280000000 -16116 7697970.840000000 -10661 7673830.200000000 -17094 7665368.160000000 -18648 7650862.020000000 -12172 7641326.400000000 -15123 7633032.500000000 -4993 7600570.800000000 -17162 7592062.560000000 -13506 7580809.830000000 -3436 7575616.330000000 -6271 7559793.930000000 -13314 7555156.630000000 -17242 7550949.500000000 -2753 7549574.060000000 -7391 7543159.680000000 -7418 7541449.650000000 -116 7520874.240000000 -12436 7520234.540000000 -1181 7494798.280000000 -12963 7491248.100000000 -213 7479470.280000000 -7114 7477681.200000000 -18521 7476478.300000000 -8973 7458603.670000000 -4202 7454095.740000000 -12009 7442105.400000000 -10609 7429346.400000000 -5622 7424142.660000000 -5143 7422760.280000000 -898 7414133.800000000 -12257 7408190.630000000 -6740 7400350.350000000 -1146 7394394.480000000 -5485 7378181.940000000 -8437 7376353.590000000 -6531 7362366.780000000 -16463 7362106.800000000 -10412 7359552.420000000 -12857 7340801.580000000 -12354 7332343.200000000 -7616 7320032.370000000 -3426 7312340.410000000 -8622 7307266.330000000 -6818 7304782.100000000 -3705 7299380.100000000 -12733 7298398.170000000 -1574 7293446.040000000 -10554 7289933.720000000 -9363 7284647.700000000 -4611 7282115.000000000 -7857 7266948.920000000 -9616 7265005.300000000 -15860 7254466.660000000 -15554 7247592.510000000 -3422 7247586.450000000 -9134 7236404.670000000 -17408 7220083.600000000 -15853 7219984.900000000 -9266 7218517.110000000 -1881 7208346.450000000 -10148 7205335.830000000 -8860 7202401.440000000 -8373 7189039.500000000 -10859 7188990.690000000 -12670 7188177.220000000 -2553 7180006.020000000 -19346 7176797.250000000 -1681 7160169.840000000 -15225 7158861.320000000 -1052 7158586.000000000 -77 7155531.100000000 -7231 7155250.380000000 -12622 7137408.420000000 -9814 7105363.140000000 -8695 7103187.000000000 -13174 7099182.530000000 -6179 7095134.050000000 -6451 7082495.360000000 -19860 7073206.830000000 -9307 7059973.680000000 -7819 7055963.040000000 -10556 7053491.070000000 -9366 7048690.740000000 -12124 7040021.310000000 -11476 7037906.760000000 -19245 7034045.240000000 -7562 7030275.790000000 -12290 7020372.060000000 -18118 7003396.800000000 -1253 7001569.620000000 -8662 6999834.270000000 -1779 6997385.730000000 -15386 6996871.790000000 -147 6989079.880000000 -9562 6983076.900000000 -3958 6969833.220000000 -7211 6966606.670000000 -12842 6923277.660000000 -4368 6918783.950000000 -11131 6918656.780000000 -4628 6894893.820000000 -5879 6881367.140000000 -16586 6865931.800000000 -32 6852925.590000000 -12119 6842773.700000000 -1371 6831137.520000000 -6136 6827917.010000000 -13857 6824240.600000000 -1074 6821747.880000000 -1863 6821522.190000000 -14597 6817385.660000000 -9271 6783068.880000000 -6389 6781075.680000000 -11703 6776538.360000000 -13701 6768880.560000000 -14880 6763788.240000000 -18428 6763670.540000000 -1006 6762065.940000000 -4927 6756765.210000000 -11659 6755246.680000000 -6815 6738928.350000000 -14367 6733857.200000000 -10703 6730936.460000000 -3150 6727920.400000000 -9963 6725919.350000000 -10438 6710153.620000000 -4745 6682153.670000000 -15297 6681711.280000000 -10848 6678666.250000000 -11749 6677895.730000000 -16739 6675549.120000000 -14915 6672248.660000000 -19841 6669191.200000000 -628 6666892.900000000 -1650 6657178.320000000 -7069 6648672.240000000 -7108 6646445.960000000 -8887 6641655.900000000 -18735 6636457.560000000 -3711 6632665.380000000 -2325 6630475.920000000 -6484 6622965.200000000 -2643 6617304.980000000 -7895 6615061.260000000 -12840 6604848.540000000 -4314 6600905.710000000 -19077 6591091.970000000 -17476 6576029.000000000 -7039 6559838.900000000 -8106 6558223.020000000 -2391 6557120.080000000 -7717 6547706.880000000 -12476 6546988.150000000 -9155 6540793.600000000 -360 6530297.410000000 -6383 6529336.020000000 -4830 6518998.920000000 -12600 6511549.460000000 -13740 6508057.920000000 -4678 6507847.980000000 -17815 6502284.760000000 -7329 6490811.950000000 -13884 6490063.100000000 -5147 6487069.000000000 -16548 6482024.500000000 -14144 6476413.400000000 -10181 6474984.880000000 -11031 6463308.020000000 -19958 6461506.380000000 -6043 6458177.640000000 -10060 6455476.890000000 -9144 6454042.050000000 -7043 6448019.980000000 -17346 6444307.520000000 -13963 6442014.480000000 -7111 6441947.070000000 -14140 6439955.540000000 -2327 6438977.280000000 -14812 6437152.540000000 -16755 6430895.360000000 -14840 6430549.140000000 -14134 6422079.260000000 -12655 6405496.790000000 -1518 6390148.220000000 -9888 6385033.020000000 -7387 6384005.180000000 -5393 6381083.040000000 -11057 6375974.220000000 -16818 6368828.800000000 -4576 6364925.710000000 -19644 6350000.330000000 -139 6336065.550000000 -11735 6334305.900000000 -10108 6332055.990000000 -15628 6329801.340000000 -4349 6314949.680000000 -7048 6313868.550000000 -17119 6298935.380000000 -18597 6291416.310000000 -2488 6286372.050000000 -2296 6275519.250000000 -4343 6272834.200000000 -9958 6267714.610000000 -2147 6267475.320000000 -9368 6254578.920000000 -13134 6252432.760000000 -10119 6251456.250000000 -2093 6249342.360000000 -2392 6237476.560000000 -17231 6233509.180000000 -3773 6226654.680000000 -9839 6214044.100000000 -19349 6213969.300000000 -7869 6212751.000000000 -8158 6210279.240000000 -13179 6205773.650000000 -2349 6205589.260000000 -9052 6200729.700000000 -1744 6189967.360000000 -597 6183103.470000000 -16721 6176606.600000000 -7498 6176277.250000000 -15157 6176222.500000000 -17524 6171107.360000000 -7922 6170906.070000000 -6615 6170738.420000000 -2706 6165991.650000000 -16432 6162740.680000000 -16473 6162427.960000000 -12825 6161595.600000000 -18813 6154678.550000000 -1030 6146500.280000000 -2571 6145772.430000000 -7707 6144754.710000000 -10327 6137612.000000000 -4710 6132346.560000000 -3649 6130602.630000000 -4893 6128461.240000000 -12844 6128191.240000000 -8794 6122690.280000000 -1157 6117749.220000000 -10895 6112017.680000000 -16166 6108250.980000000 -11920 6107122.560000000 -5621 6102123.560000000 -6141 6093826.560000000 -14076 6086671.080000000 -15884 6080485.590000000 -4814 6080337.960000000 -5814 6079842.960000000 -1134 6078685.200000000 -19048 6071813.280000000 -303 6070601.350000000 -15211 6065678.060000000 -1020 6054168.000000000 -11292 6052522.830000000 -7134 6049865.250000000 -14119 6049619.550000000 -2389 6042429.120000000 -5926 6034269.520000000 -8553 6030922.950000000 -18814 6023255.170000000 -12702 6023190.990000000 -2644 6020931.330000000 -19628 6010704.000000000 -18112 6008707.520000000 -13860 6008170.290000000 -1456 6005092.140000000 -1633 6002068.850000000 -2301 6000302.290000000 -10740 5999809.050000000 -2630 5997008.430000000 -8818 5992296.900000000 -10043 5990594.020000000 -653 5987942.830000000 -6829 5985990.660000000 -15179 5977727.520000000 -9663 5973523.920000000 -5863 5973328.920000000 -3628 5966340.090000000 -7618 5960155.860000000 -2588 5952648.560000000 -4865 5949383.400000000 -4233 5944699.600000000 -13390 5944104.690000000 -16321 5942714.700000000 -9653 5941308.500000000 -18884 5925548.240000000 -4394 5920927.140000000 -19774 5916723.120000000 -1257 5914052.360000000 -2963 5911917.770000000 -17157 5899573.020000000 -383 5884693.360000000 -11709 5884134.480000000 -18135 5871431.610000000 -13169 5869736.880000000 -2932 5868995.800000000 -2888 5863229.300000000 -6820 5853106.800000000 -18622 5850951.600000000 -9066 5846052.720000000 -19840 5832251.400000000 -6325 5827298.430000000 -14092 5823711.720000000 -11532 5823277.120000000 -18043 5815248.700000000 -3932 5809134.600000000 -10364 5808371.400000000 -1179 5808303.240000000 -11441 5799378.330000000 -15767 5798263.260000000 -14063 5797204.000000000 -11383 5793893.120000000 -10065 5781764.450000000 -17785 5766838.760000000 -18349 5761765.770000000 -14824 5760339.780000000 -14699 5759490.180000000 -11628 5755137.200000000 -4403 5752376.640000000 -13090 5751949.250000000 -15332 5744606.280000000 -17573 5744384.150000000 -12895 5741007.020000000 -13577 5739209.940000000 -16594 5732523.640000000 -8497 5727509.840000000 -2872 5724068.720000000 -16181 5721788.160000000 -6639 5712041.220000000 -13095 5708798.250000000 -4535 5693341.200000000 -10956 5692585.500000000 -19755 5686913.370000000 -12995 5682200.410000000 -13157 5681454.940000000 -1887 5681341.240000000 -18472 5680356.000000000 -19814 5679360.000000000 -18583 5669360.520000000 -3894 5664385.490000000 -1740 5659552.040000000 -62 5659470.160000000 -16532 5653779.460000000 -995 5648973.450000000 -7486 5646851.870000000 -19007 5642710.930000000 -13060 5642024.640000000 -12371 5635710.450000000 -2280 5634077.400000000 -3115 5631352.320000000 -11107 5631252.460000000 -5873 5629125.750000000 -14743 5628732.800000000 -2224 5624746.620000000 -2653 5623859.990000000 -17622 5623057.440000000 -14700 5615344.000000000 -14740 5613929.390000000 -6403 5611924.510000000 -6896 5609169.880000000 -10384 5607337.520000000 -16433 5605707.030000000 -5898 5604937.760000000 -4789 5600488.360000000 -8132 5593107.500000000 -3838 5592313.200000000 -13631 5586424.800000000 -11233 5585247.010000000 -849 5583516.450000000 -14653 5581550.450000000 -14788 5580433.000000000 -18181 5578562.880000000 -19815 5577102.620000000 -5584 5576692.200000000 -5385 5576420.190000000 -13780 5569028.610000000 -9342 5566783.710000000 -19056 5566524.120000000 -8189 5565694.740000000 -13808 5560721.960000000 -10635 5560058.550000000 -8304 5550784.410000000 -14257 5549164.060000000 -8999 5542100.100000000 -19134 5539312.560000000 -8360 5538031.100000000 -6397 5536651.920000000 -2597 5525317.760000000 -8631 5515909.380000000 -16729 5512663.650000000 -11861 5511785.930000000 -16853 5511689.910000000 -6341 5502790.080000000 -1312 5496649.400000000 -5566 5495885.870000000 -12519 5490649.970000000 -19032 5488105.020000000 -8231 5479312.500000000 -3026 5466732.320000000 -6388 5466168.800000000 -15349 5464571.520000000 -18985 5463897.130000000 -19848 5454266.400000000 -17378 5453284.740000000 -4000 5448690.390000000 -3710 5445822.530000000 -13181 5439774.060000000 -6420 5438325.320000000 -3644 5437772.140000000 -1117 5437024.970000000 -13027 5436968.460000000 -12884 5432632.340000000 -12781 5429161.080000000 -10084 5428231.620000000 -5640 5423318.580000000 -16208 5422901.400000000 -753 5416899.740000000 -4120 5413822.460000000 -12348 5412061.710000000 -1586 5411198.550000000 -2503 5411125.800000000 -1381 5397790.100000000 -19673 5397746.770000000 -19389 5394426.400000000 -15532 5386361.380000000 -3562 5380335.080000000 -19114 5375133.000000000 -3204 5372215.460000000 -6653 5365178.000000000 -3553 5363690.290000000 -12687 5361682.120000000 -3279 5357505.650000000 -9840 5350112.180000000 -8258 5347064.670000000 -11387 5345284.000000000 -15210 5341117.920000000 -15092 5340896.800000000 -6052 5339762.180000000 -14043 5339587.680000000 -6284 5336153.000000000 -6297 5332357.740000000 -16254 5326607.830000000 -18436 5326198.500000000 -14436 5325517.140000000 -10368 5319983.080000000 -6349 5317943.400000000 -19984 5317620.000000000 -19080 5310678.600000000 -1403 5306771.320000000 -5995 5305056.120000000 -13214 5299420.450000000 -19293 5297054.400000000 -7130 5289761.480000000 -9809 5286784.800000000 -9273 5277924.770000000 -16786 5260035.000000000 -11032 5256354.350000000 -17719 5246322.200000000 -3400 5244203.680000000 -8278 5243718.480000000 -7873 5241168.800000000 -9357 5239695.770000000 -2239 5235767.830000000 -18695 5223960.350000000 -19271 5216498.290000000 -4563 5214673.700000000 -1896 5210699.790000000 -15755 5210481.240000000 -17628 5209882.370000000 -5747 5206892.240000000 -18057 5204907.400000000 -5075 5204412.810000000 -5325 5190162.750000000 -17675 5183135.360000000 -18818 5181295.300000000 -11064 5180667.570000000 -12578 5177222.330000000 -3725 5169183.750000000 -11112 5161546.000000000 -4749 5161499.440000000 -7632 5159180.500000000 -13119 5151665.510000000 -5653 5140324.640000000 -16134 5137482.510000000 -5671 5136815.550000000 -18348 5132981.070000000 -562 5132487.250000000 -2562 5126763.830000000 -10304 5125180.000000000 -7622 5120719.980000000 -12755 5118213.920000000 -1814 5100705.580000000 -11269 5096671.330000000 -2964 5095253.720000000 -1616 5091834.000000000 -9294 5090753.530000000 -16793 5085330.540000000 -4999 5081651.750000000 -17209 5079029.280000000 -7151 5078937.600000000 -15522 5072469.600000000 -3056 5072329.550000000 -7612 5068322.870000000 -18453 5063892.920000000 -18324 5058901.220000000 -12266 5058186.750000000 -19394 5056235.730000000 -1713 5054968.050000000 -15681 5051569.630000000 -8274 5043328.000000000 -18160 5043074.830000000 -18253 5041572.000000000 -11840 5040590.040000000 -1532 5033171.000000000 -584 5031602.640000000 -12382 5028901.000000000 -14814 5022200.070000000 -19058 5019001.920000000 -4487 5016640.860000000 -8482 5015444.250000000 -18476 5011136.360000000 -12335 5003581.400000000 -4455 4997933.310000000 -14355 4992822.920000000 -15253 4992642.200000000 -14069 4983244.200000000 -17843 4977294.370000000 -9389 4975885.830000000 -14435 4971442.190000000 -13254 4959481.450000000 -9773 4955887.800000000 -7615 4952421.540000000 -6476 4947250.050000000 -9253 4945159.700000000 -14932 4934395.480000000 -13253 4932867.450000000 -19322 4931525.780000000 -16945 4931440.610000000 -731 4930191.930000000 -6540 4925114.510000000 -5148 4923048.000000000 -1934 4921196.900000000 -15402 4920840.720000000 -17914 4919607.040000000 -5416 4916041.920000000 -16734 4914205.270000000 -14967 4900262.080000000 -8706 4894595.580000000 -136 4891960.920000000 -19494 4886028.300000000 -8737 4880640.440000000 -7653 4879423.640000000 -4149 4875782.400000000 -7890 4872424.400000000 -11142 4871415.420000000 -10386 4863623.500000000 -8603 4861814.610000000 -2680 4861223.740000000 -4891 4858103.880000000 -19236 4855097.690000000 -14251 4854739.860000000 -18895 4853365.100000000 -17134 4852041.040000000 -4932 4843701.450000000 -10033 4841647.720000000 -1383 4839144.310000000 -18721 4837983.360000000 -8618 4833125.420000000 -17386 4831545.840000000 -3790 4830282.360000000 -1043 4825921.310000000 -12434 4822597.780000000 -18385 4819643.400000000 -6046 4817460.060000000 -5821 4814423.450000000 -10836 4814303.240000000 -6848 4813954.080000000 -6880 4804600.350000000 -11249 4800116.820000000 -11970 4799739.660000000 -14253 4796521.290000000 -7782 4793227.130000000 -75 4790042.880000000 -7076 4789347.340000000 -9566 4782531.800000000 -2137 4767931.740000000 -2336 4763870.790000000 -15362 4759043.380000000 -3284 4755048.760000000 -12964 4753627.480000000 -1781 4752835.200000000 -5454 4748342.980000000 -12597 4742077.840000000 -19120 4733459.960000000 -14884 4731499.440000000 -825 4730720.280000000 -14683 4730482.320000000 -5361 4726113.000000000 -12179 4725018.100000000 -1461 4710954.690000000 -9890 4709658.400000000 -13369 4705085.390000000 -11612 4701627.990000000 -3096 4699414.400000000 -10383 4697866.470000000 -11293 4697336.060000000 -3383 4695825.200000000 -6231 4694381.720000000 -7396 4691319.060000000 -17827 4688797.440000000 -15856 4683368.210000000 -8253 4678560.860000000 -12327 4677984.740000000 -4395 4676829.820000000 -4232 4676646.400000000 -14260 4670522.800000000 -15288 4669273.990000000 -17526 4668545.640000000 -9884 4662693.840000000 -2118 4660352.780000000 -4524 4653956.600000000 -19090 4650872.940000000 -3928 4649359.440000000 -14325 4647762.170000000 -15476 4643469.040000000 -4179 4639931.760000000 -14408 4639631.280000000 -19424 4634817.440000000 -3334 4633102.500000000 -9477 4628073.510000000 -11803 4625396.800000000 -14805 4618452.180000000 -463 4616307.280000000 -16628 4607490.960000000 -3116 4604463.100000000 -19962 4602949.470000000 -12859 4602870.550000000 -12063 4600708.450000000 -5648 4592273.250000000 -8556 4590726.860000000 -15281 4589425.410000000 -9414 4587426.900000000 -13951 4586281.250000000 -19328 4582624.820000000 -15963 4579705.500000000 -10773 4573276.200000000 -14179 4568816.000000000 -1895 4563988.160000000 -6408 4561496.390000000 -5958 4554000.000000000 -3653 4548134.400000000 -11218 4546237.920000000 -19327 4543987.770000000 -9572 4535941.160000000 -14556 4531464.750000000 -2475 4529761.500000000 -9631 4529261.560000000 -1901 4528592.550000000 -86 4528475.380000000 -9586 4527146.220000000 -17361 4519098.870000000 -8112 4514949.450000000 -13468 4499728.200000000 -18239 4497633.640000000 -10215 4494553.600000000 -6211 4492264.960000000 -836 4490945.100000000 -895 4489141.500000000 -19542 4488393.750000000 -4322 4487884.230000000 -2116 4486944.650000000 -553 4486075.480000000 -2515 4485188.260000000 -16286 4481470.470000000 -12271 4478224.950000000 -16570 4465818.000000000 -7995 4457574.660000000 -18396 4457229.600000000 -16331 4455735.480000000 -18157 4452196.630000000 -5271 4452040.010000000 -11622 4451244.840000000 -4052 4446397.340000000 -2864 4446008.380000000 -490 4442892.300000000 -19837 4434172.390000000 -4114 4433657.850000000 -11436 4433070.150000000 -6085 4431306.570000000 -9735 4430445.600000000 -17834 4416286.330000000 -8157 4416116.650000000 -18840 4414925.320000000 -13553 4412261.700000000 -12562 4411183.040000000 -14025 4403442.160000000 -17964 4400360.090000000 -636 4399863.840000000 -8390 4389024.330000000 -231 4387397.300000000 -9699 4385891.020000000 -10622 4384005.320000000 -14364 4383236.900000000 -10580 4381533.230000000 -10124 4369800.960000000 -10451 4368867.500000000 -4673 4367113.440000000 -11351 4362616.500000000 -4770 4362397.320000000 -12932 4362042.600000000 -10603 4357216.500000000 -19733 4348931.750000000 -4222 4348871.910000000 -17319 4347687.690000000 -3375 4346529.480000000 -14995 4338295.650000000 -7675 4337499.600000000 -15043 4333921.200000000 -4835 4332648.000000000 -4408 4332588.900000000 -5559 4330577.090000000 -7376 4328936.540000000 -18061 4328793.980000000 -2749 4328671.530000000 -6628 4328501.880000000 -5888 4323049.720000000 -18872 4322595.620000000 -5476 4319642.580000000 -1755 4318935.630000000 -10623 4315822.560000000 -18775 4314677.640000000 -3570 4312697.870000000 -11147 4310740.570000000 -6071 4307612.400000000 -10807 4306006.000000000 -9550 4299478.560000000 -657 4296794.190000000 -19669 4294640.900000000 -8532 4290651.600000000 -13469 4281715.620000000 -8809 4280778.800000000 -11301 4276847.950000000 -6147 4266879.920000000 -2612 4265962.350000000 -15699 4256118.720000000 -12300 4254409.110000000 -3494 4250810.600000000 -11040 4250030.200000000 -6190 4244046.800000000 -17616 4239937.500000000 -7271 4234407.000000000 -14048 4226977.440000000 -4456 4224684.980000000 -10012 4223841.210000000 -11175 4223704.140000000 -18675 4215406.860000000 -10792 4214898.570000000 -10806 4209678.320000000 -18749 4204787.000000000 -17410 4198025.280000000 -8032 4195430.000000000 -11094 4192304.940000000 -17582 4187341.440000000 -12246 4183230.950000000 -6640 4182968.800000000 -7346 4174707.600000000 -12747 4169865.810000000 -3869 4164957.440000000 -13106 4161902.080000000 -10547 4159541.360000000 -15289 4156205.760000000 -1679 4156156.640000000 -1126 4155593.080000000 -19106 4147439.520000000 -9705 4144024.200000000 -15324 4142518.560000000 -16544 4140375.720000000 -8812 4139322.810000000 -10772 4134101.640000000 -2800 4127150.080000000 -15549 4124704.640000000 -3607 4118697.570000000 -1980 4117633.720000000 -214 4113117.360000000 -19217 4104217.600000000 -2460 4098577.460000000 -19156 4093864.460000000 -18359 4092727.290000000 -12865 4092526.840000000 -14616 4092434.540000000 -908 4088856.200000000 -11791 4083804.970000000 -4157 4078345.600000000 -3857 4070872.870000000 -15114 4056112.500000000 -395 4052997.760000000 -17456 4051457.280000000 -10562 4050894.190000000 -10884 4050330.760000000 -12177 4049842.680000000 -15595 4040577.560000000 -15916 4036044.500000000 -7084 4035102.720000000 -4424 4034761.560000000 -10874 4031015.850000000 -4740 4030403.760000000 -16585 4030010.260000000 -18824 4028984.100000000 -14875 4028452.080000000 -13855 4024828.340000000 -10932 4024002.400000000 -9084 4021362.450000000 -14352 4018089.740000000 -18086 4015180.680000000 -9514 4013666.670000000 -15787 4013154.560000000 -714 4010249.440000000 -8811 4009588.900000000 -14386 4007210.880000000 -616 4004057.260000000 -7460 4003412.480000000 -866 4003182.540000000 -782 4001299.940000000 -8562 3999441.620000000 -1366 3994060.860000000 -2879 3993056.550000000 -16679 3992434.990000000 -17306 3990723.300000000 -13140 3982817.390000000 -17942 3980857.040000000 -6572 3977676.280000000 -3578 3977523.940000000 -15802 3969946.900000000 -336 3967938.380000000 -9807 3964469.600000000 -12104 3964273.400000000 -4271 3962359.280000000 -6702 3961657.440000000 -19763 3955582.750000000 -369 3953702.880000000 -4089 3953455.680000000 -2593 3946153.800000000 -590 3943841.160000000 -8325 3942118.750000000 -158 3941881.650000000 -12054 3938362.690000000 -18330 3938303.880000000 -5354 3936239.580000000 -8150 3925793.460000000 -8344 3921293.600000000 -6069 3921130.550000000 -4032 3920008.590000000 -17939 3917750.270000000 -7014 3914471.200000000 -2840 3913131.580000000 -1868 3912987.540000000 -10975 3911920.480000000 -5374 3910802.740000000 -11128 3908156.460000000 -18449 3907589.400000000 -11740 3907459.840000000 -2356 3907189.000000000 -5721 3901585.970000000 -4231 3900779.050000000 -4352 3899933.440000000 -432 3899836.440000000 -15321 3899516.580000000 -10296 3897015.140000000 -5647 3895088.160000000 -7386 3891916.510000000 -507 3891487.680000000 -3995 3887387.070000000 -4278 3882294.020000000 -18407 3880267.860000000 -6127 3879166.710000000 -145 3875277.240000000 -19269 3874685.760000000 -18257 3874454.890000000 -9068 3869767.740000000 -576 3860007.790000000 -4860 3852862.020000000 -18793 3849838.160000000 -15988 3847257.050000000 -6891 3846386.750000000 -3231 3846344.300000000 -15237 3845421.000000000 -9035 3844166.850000000 -7597 3838643.350000000 -16349 3837121.650000000 -2497 3827850.200000000 -3616 3827390.950000000 -11566 3826122.470000000 -18403 3822033.020000000 -2972 3821903.550000000 -812 3821523.720000000 -2043 3820561.360000000 -505 3818922.030000000 -8257 3815071.920000000 -6084 3814194.950000000 -11253 3813917.240000000 -366 3812257.880000000 -13632 3811601.320000000 -14298 3801412.420000000 -7092 3798729.480000000 -2058 3796109.040000000 -14820 3791195.860000000 -7157 3788690.820000000 -17211 3786030.170000000 -16644 3786019.250000000 -15693 3783662.190000000 -2627 3782394.600000000 -11231 3782077.600000000 -12696 3781761.660000000 -8705 3778077.000000000 -16052 3771577.040000000 -99 3760269.310000000 -2082 3757517.500000000 -872 3750005.340000000 -7126 3749138.920000000 -10302 3744475.250000000 -17122 3741012.980000000 -10080 3740107.100000000 -16021 3739611.200000000 -3074 3739224.960000000 -3142 3738811.020000000 -13213 3735116.250000000 -13442 3733132.140000000 -11542 3731000.120000000 -13732 3730444.900000000 -2608 3729372.400000000 -5 3725511.500000000 -19157 3723844.720000000 -18231 3721707.990000000 -8179 3714155.040000000 -12740 3708646.910000000 -11597 3706528.590000000 -13968 3702376.080000000 -6436 3687346.440000000 -9181 3687134.080000000 -564 3680200.800000000 -13464 3678406.200000000 -14084 3673790.380000000 -2755 3670593.690000000 -14284 3668640.800000000 -12178 3653392.480000000 -15730 3650258.300000000 -5560 3649569.590000000 -8594 3647140.560000000 -7032 3646439.540000000 -16846 3644843.100000000 -1530 3642838.080000000 -3978 3639712.050000000 -2897 3639442.320000000 -16625 3636527.540000000 -12029 3636339.720000000 -16830 3633448.570000000 -9597 3632662.110000000 -5533 3630338.670000000 -5181 3625965.930000000 -8131 3625738.620000000 -8560 3620761.260000000 -11860 3618746.250000000 -12008 3614604.400000000 -10737 3611990.640000000 -18208 3611596.100000000 -5119 3611038.200000000 -11958 3601654.650000000 -15124 3598278.200000000 -14058 3597490.020000000 -12270 3593912.100000000 -17793 3593318.950000000 -9385 3587327.840000000 -12814 3587083.840000000 -5304 3586230.610000000 -3631 3582841.650000000 -610 3581917.300000000 -19317 3580412.430000000 -128 3567004.560000000 -11616 3566154.800000000 -10176 3565392.150000000 -7349 3564110.640000000 -1712 3560408.430000000 -18860 3559340.600000000 -17617 3557516.000000000 -6443 3556296.960000000 -15408 3554814.560000000 -16350 3554388.630000000 -17436 3554105.130000000 -5740 3551324.680000000 -12181 3550218.540000000 -16895 3550119.300000000 -19995 3548839.700000000 -4968 3548306.870000000 -2257 3546692.290000000 -1825 3543198.780000000 -18989 3539038.080000000 -18727 3536081.400000000 -16165 3533789.840000000 -3249 3533709.870000000 -11731 3532875.000000000 -13032 3532415.790000000 -9377 3531582.080000000 -5883 3531479.000000000 -1211 3528833.400000000 -12065 3526948.100000000 -10866 3526146.660000000 -2073 3520131.300000000 -2378 3512186.200000000 -16860 3509693.070000000 -389 3507814.640000000 -15604 3505653.270000000 -11257 3502831.800000000 -1327 3502022.600000000 -16602 3501074.880000000 -1493 3498808.950000000 -8224 3498179.520000000 -622 3497158.360000000 -3072 3495958.720000000 -1478 3494880.480000000 -3125 3494169.900000000 -2052 3488438.080000000 -8476 3487191.280000000 -10735 3477740.760000000 -14860 3476235.840000000 -6586 3475745.100000000 -5130 3472024.500000000 -7181 3471306.300000000 -618 3467906.520000000 -15698 3464859.470000000 -17585 3462450.460000000 -2548 3456856.960000000 -2632 3456230.740000000 -2882 3453986.860000000 -12216 3452907.150000000 -4925 3452904.630000000 -9012 3442581.360000000 -6667 3430076.400000000 -17958 3424962.560000000 -6093 3424241.920000000 -10648 3417414.000000000 -1462 3413248.610000000 -2569 3412388.820000000 -18616 3409880.910000000 -7368 3408036.450000000 -3110 3407374.600000000 -10824 3406819.290000000 -11510 3404701.960000000 -4840 3397236.400000000 -4449 3396993.600000000 -1358 3396616.320000000 -3885 3395817.600000000 -13381 3391953.520000000 -1655 3383051.510000000 -282 3381785.420000000 -4928 3374270.480000000 -3199 3372488.800000000 -16086 3370710.650000000 -8612 3362922.500000000 -19597 3360764.000000000 -8867 3354400.110000000 -4098 3353574.280000000 -12617 3351499.050000000 -14365 3347296.000000000 -10443 3345493.100000000 -76 3342081.820000000 -11585 3341941.220000000 -4383 3338960.270000000 -13910 3335964.160000000 -8076 3332449.890000000 -16005 3332190.400000000 -2622 3329364.450000000 -12822 3321183.520000000 -17076 3320398.060000000 -5392 3320357.150000000 -18628 3319615.840000000 -13695 3318525.990000000 -10326 3318274.160000000 -9109 3317833.900000000 -1489 3317620.800000000 -3378 3315948.000000000 -7738 3312979.200000000 -1844 3312277.360000000 -19963 3307500.000000000 -2436 3306419.050000000 -886 3302180.700000000 -15475 3301693.500000000 -6327 3300680.780000000 -6050 3299460.200000000 -9876 3298410.050000000 -19586 3291131.250000000 -14349 3289862.520000000 -10993 3287980.570000000 -18784 3286752.120000000 -1800 3285466.240000000 -990 3284595.500000000 -3823 3281992.940000000 -15737 3279305.960000000 -19518 3276759.630000000 -9032 3272440.320000000 -7786 3271217.280000000 -8648 3271162.440000000 -5532 3270187.970000000 -15914 3268520.980000000 -16065 3265068.840000000 -11212 3264657.030000000 -13229 3262022.280000000 -15827 3260862.720000000 -1582 3260340.000000000 -3827 3260093.760000000 -3546 3259244.070000000 -15849 3258918.000000000 -14856 3258379.400000000 -2028 3255013.960000000 -6618 3254581.950000000 -17461 3252926.880000000 -13551 3241602.200000000 -19561 3239795.320000000 -2276 3236172.300000000 -14203 3234649.390000000 -7757 3231351.840000000 -122 3226213.880000000 -12954 3225943.000000000 -647 3224783.760000000 -12383 3223989.440000000 -3831 3223126.600000000 -16836 3222260.730000000 -4565 3221597.440000000 -19426 3218106.540000000 -17855 3217813.020000000 -5624 3207777.360000000 -8368 3203376.450000000 -9480 3200904.000000000 -11181 3199500.530000000 -8981 3197864.000000000 -16426 3195995.970000000 -1648 3195558.900000000 -14404 3192729.600000000 -17867 3188571.000000000 -18117 3183229.040000000 -14289 3182261.600000000 -53 3182256.000000000 -15546 3180180.040000000 -16245 3178277.460000000 -1597 3176247.480000000 -1653 3173456.640000000 -2845 3171619.610000000 -15906 3171187.540000000 -18304 3168571.500000000 -14068 3167367.600000000 -6837 3165012.480000000 -9446 3164446.520000000 -18889 3156140.960000000 -16587 3154210.200000000 -7705 3152977.380000000 -1120 3151591.170000000 -17665 3148848.000000000 -5311 3146721.860000000 -14157 3144707.320000000 -7996 3131351.040000000 -8663 3130526.320000000 -18271 3127800.960000000 -6446 3125685.960000000 -6972 3125007.060000000 -2572 3123186.830000000 -13536 3122527.540000000 -6196 3122172.480000000 -9338 3121262.400000000 -11992 3118647.550000000 -2580 3118284.370000000 -9098 3117494.100000000 -5118 3112661.960000000 -10184 3109293.400000000 -9932 3105818.240000000 -18545 3102273.320000000 -10963 3099314.500000000 -8405 3097121.120000000 -9037 3095195.000000000 -179 3091107.280000000 -1930 3090915.800000000 -17723 3090624.660000000 -4308 3089472.750000000 -8702 3080129.920000000 -18621 3079984.800000000 -4501 3079781.100000000 -3590 3079049.420000000 -18264 3078858.440000000 -15648 3078564.060000000 -5998 3073264.000000000 -16904 3072610.800000000 -3794 3071333.090000000 -3147 3068485.320000000 -17221 3068337.220000000 -4709 3067523.310000000 -18017 3066743.410000000 -15613 3063987.860000000 -16271 3057051.340000000 -13621 3054774.590000000 -12919 3054518.500000000 -12493 3050836.300000000 -15838 3050645.950000000 -3273 3048955.150000000 -8324 3046011.250000000 -13628 3045324.500000000 -5522 3044408.500000000 -2202 3043132.050000000 -19052 3042566.550000000 -5767 3041871.700000000 -17895 3036452.220000000 -12586 3036386.300000000 -12425 3035041.520000000 -13517 3034351.470000000 -2363 3033336.600000000 -15060 3032598.510000000 -6764 3032591.100000000 -340 3030522.000000000 -4723 3028910.250000000 -3566 3027858.610000000 -17796 3026838.960000000 -15384 3023792.640000000 -16336 3010813.560000000 -679 3010713.300000000 -7554 3010667.800000000 -14553 3009756.960000000 -8379 3009745.170000000 -15436 3007499.770000000 -12471 3003991.860000000 -18059 3003037.530000000 -8536 3000746.000000000 -19033 2999373.280000000 -18179 2996151.200000000 -10711 2996143.170000000 -17271 2994264.790000000 -13932 2989023.580000000 -3101 2987788.160000000 -14550 2977853.650000000 -3080 2977232.580000000 -14533 2976490.490000000 -14439 2975313.240000000 -9237 2973124.780000000 -1205 2971470.280000000 -12361 2963419.470000000 -429 2962631.880000000 -3970 2960418.450000000 -8403 2957698.450000000 -1098 2957514.000000000 -7932 2955046.140000000 -16266 2952298.380000000 -19386 2948854.480000000 -13147 2947037.910000000 -2720 2947011.080000000 -3840 2944219.350000000 -13482 2942474.880000000 -9436 2940396.210000000 -19779 2937105.960000000 -18032 2933224.380000000 -7743 2932733.770000000 -14620 2930766.890000000 -4606 2927832.590000000 -18076 2924134.830000000 -19276 2918176.200000000 -7483 2915918.950000000 -8575 2915132.640000000 -11052 2913140.880000000 -17251 2908345.800000000 -8788 2907935.930000000 -10960 2906511.140000000 -18782 2903643.780000000 -19988 2897461.530000000 -726 2896009.270000000 -19942 2894251.360000000 -10864 2892252.480000000 -17840 2891563.220000000 -18717 2888939.960000000 -12391 2886051.300000000 -18219 2885921.060000000 -15100 2883342.330000000 -2491 2880385.740000000 -12389 2879696.960000000 -3880 2877770.240000000 -18579 2874542.480000000 -13647 2873838.340000000 -15758 2873804.920000000 -12917 2873659.600000000 -18866 2873616.260000000 -13894 2872986.120000000 -15200 2872571.930000000 -9628 2872404.560000000 -8568 2871598.080000000 -8389 2870237.880000000 -5788 2867210.180000000 -19450 2863310.660000000 -9440 2863162.920000000 -16795 2860135.410000000 -19643 2858987.800000000 -1974 2856825.840000000 -14622 2852089.120000000 -6885 2851437.620000000 -12532 2848992.640000000 -1087 2847858.800000000 -5777 2846407.410000000 -5629 2846076.120000000 -6316 2840544.650000000 -12767 2840514.120000000 -12134 2840036.910000000 -14476 2839853.010000000 -803 2838388.160000000 -18964 2836942.440000000 -6020 2833459.200000000 -10401 2832688.740000000 -1323 2829964.500000000 -1151 2829662.440000000 -1458 2824034.430000000 -2271 2820756.530000000 -18740 2814140.800000000 -7348 2811730.950000000 -4281 2807190.520000000 -8043 2804706.240000000 -3843 2804217.960000000 -7813 2802350.880000000 -347 2802245.520000000 -745 2801725.100000000 -10388 2799170.580000000 -18100 2793358.500000000 -19043 2789013.800000000 -10644 2787797.010000000 -16170 2787402.800000000 -398 2782729.050000000 -9370 2780078.130000000 -14504 2780036.040000000 -1210 2778485.760000000 -13385 2777445.620000000 -3799 2775223.600000000 -11325 2769766.020000000 -3489 2769554.520000000 -17181 2769028.500000000 -6964 2766653.780000000 -7381 2764898.800000000 -6253 2764394.640000000 -5975 2760819.720000000 -11996 2760687.860000000 -7570 2758977.120000000 -4387 2757672.000000000 -9014 2755367.420000000 -9403 2748021.660000000 -11653 2739731.070000000 -17697 2739312.290000000 -958 2738032.000000000 -18816 2737140.000000000 -14104 2735008.640000000 -15966 2732250.200000000 -17912 2724160.950000000 -7089 2720170.040000000 -16032 2718976.180000000 -16891 2717293.320000000 -19579 2716909.860000000 -17470 2715048.840000000 -12408 2712556.520000000 -4763 2711800.900000000 -1138 2709709.810000000 -7363 2708414.400000000 -7877 2705439.450000000 -17532 2703698.680000000 -10512 2701235.920000000 -11957 2700133.220000000 -2455 2699593.880000000 -15119 2696860.800000000 -9868 2696801.520000000 -14172 2695307.480000000 -16120 2689337.820000000 -13958 2679025.280000000 -15169 2676686.040000000 -2648 2672232.000000000 -6164 2671317.320000000 -12701 2669216.400000000 -16382 2669034.540000000 -15588 2667212.100000000 -14830 2666758.150000000 -9119 2665812.240000000 -1622 2665206.500000000 -878 2664045.790000000 -13269 2662784.120000000 -619 2655417.630000000 -18386 2653795.020000000 -2501 2652260.400000000 -2310 2651631.090000000 -19420 2649395.610000000 -4895 2645152.270000000 -7553 2643682.070000000 -17814 2642781.440000000 -16097 2642500.000000000 -10995 2640811.160000000 -14895 2637733.720000000 -18546 2637026.710000000 -9875 2631358.800000000 -9591 2626899.540000000 -6001 2625893.760000000 -7739 2624573.280000000 -10431 2624379.540000000 -4544 2615313.750000000 -16387 2603195.760000000 -18375 2601407.830000000 -8395 2598728.440000000 -18853 2593356.360000000 -4900 2592813.150000000 -1302 2592197.760000000 -17032 2589806.400000000 -14292 2589749.560000000 -43 2587359.580000000 -5221 2587024.040000000 -397 2579751.460000000 -17890 2579674.240000000 -12157 2575510.480000000 -7340 2574645.830000000 -19368 2572618.950000000 -8848 2570819.150000000 -13789 2570243.260000000 -14596 2568234.240000000 -8408 2567434.410000000 -19726 2565750.420000000 -13964 2565579.120000000 -7740 2563027.500000000 -14768 2560392.600000000 -11734 2559062.220000000 -10294 2558257.970000000 -15032 2557926.220000000 -9127 2556379.800000000 -2181 2553175.000000000 -16653 2552229.680000000 -3866 2549994.790000000 -16814 2548710.760000000 -1866 2545838.400000000 -3512 2532626.800000000 -4145 2529786.150000000 -12120 2528298.720000000 -644 2528123.050000000 -15379 2525181.010000000 -6392 2524063.080000000 -2652 2521456.800000000 -3363 2519202.230000000 -19167 2517993.180000000 -16042 2516599.920000000 -2892 2511854.400000000 -5711 2509401.720000000 -14591 2506344.690000000 -6564 2506277.340000000 -1231 2505421.240000000 -5049 2502603.000000000 -14576 2501606.690000000 -10211 2500852.200000000 -293 2493168.480000000 -7371 2491134.650000000 -18154 2491047.200000000 -9494 2489825.520000000 -14836 2480432.400000000 -19471 2480403.750000000 -802 2478998.330000000 -12541 2477242.600000000 -15065 2473563.940000000 -15995 2472803.200000000 -9408 2471953.560000000 -9776 2470447.900000000 -17325 2468989.050000000 -3391 2468317.720000000 -16123 2467022.220000000 -18758 2463798.060000000 -407 2460304.470000000 -6840 2456170.780000000 -9995 2455155.360000000 -3877 2453696.650000000 -5817 2452493.130000000 -14122 2452226.220000000 -16699 2450273.980000000 -8921 2450116.480000000 -15103 2449861.200000000 -7637 2449628.720000000 -3076 2443927.380000000 -6648 2443248.950000000 -17116 2442263.720000000 -1645 2440838.400000000 -3181 2440017.600000000 -5966 2431558.080000000 -15882 2428947.300000000 -7529 2428381.280000000 -12836 2427897.330000000 -18052 2427637.760000000 -13616 2426638.500000000 -16615 2424775.080000000 -18147 2424412.680000000 -4586 2424123.900000000 -14403 2423141.960000000 -11606 2422794.310000000 -13526 2422212.800000000 -3677 2421404.460000000 -5553 2418506.210000000 -12109 2416514.170000000 -13118 2415931.800000000 -1563 2408855.400000000 -16591 2408045.390000000 -6411 2404918.530000000 -10272 2402834.480000000 -10597 2400247.680000000 -13700 2398035.860000000 -9548 2397147.900000000 -14963 2395781.090000000 -13325 2390637.580000000 -13864 2388067.880000000 -7450 2383447.710000000 -9275 2382868.400000000 -5829 2378037.920000000 -13437 2377806.540000000 -13594 2375046.300000000 -11442 2374591.080000000 -15619 2374052.380000000 -9063 2374035.840000000 -5990 2368686.500000000 -7811 2363829.260000000 -9525 2362974.530000000 -5597 2361031.840000000 -8963 2360774.000000000 -1709 2359839.290000000 -15814 2358656.640000000 -17613 2357519.040000000 -5022 2354550.450000000 -17740 2354242.830000000 -3388 2351042.260000000 -13773 2348739.120000000 -14467 2348665.040000000 -11544 2345324.450000000 -349 2344664.130000000 -10356 2340862.720000000 -18272 2338754.600000000 -4627 2337430.840000000 -327 2335298.460000000 -19846 2332224.730000000 -10814 2330319.600000000 -13102 2326122.750000000 -18867 2323972.000000000 -2824 2323315.080000000 -19117 2319911.100000000 -1906 2319757.600000000 -245 2319450.900000000 -17318 2317860.390000000 -3862 2316453.720000000 -8100 2313874.120000000 -2958 2312239.470000000 -10263 2308514.060000000 -13814 2304940.400000000 -9394 2303161.740000000 -18080 2299416.780000000 -1271 2289526.980000000 -3327 2278474.480000000 -8740 2278405.920000000 -8119 2276428.170000000 -3368 2274373.620000000 -7963 2272300.800000000 -2151 2270932.720000000 -16995 2270264.680000000 -9918 2269733.070000000 -503 2268535.250000000 -16692 2256484.500000000 -793 2254198.720000000 -16455 2252361.860000000 -6644 2249521.820000000 -17280 2249437.500000000 -6813 2248982.000000000 -4674 2246915.320000000 -16325 2244369.800000000 -182 2243290.000000000 -4626 2242474.350000000 -10860 2241291.600000000 -14034 2241220.800000000 -2476 2240855.200000000 -4253 2239985.640000000 -3211 2239871.020000000 -1290 2233313.000000000 -8479 2232189.040000000 -11895 2231607.000000000 -3487 2230171.620000000 -14870 2229915.370000000 -16328 2229483.960000000 -18585 2228215.500000000 -7638 2228208.080000000 -5436 2225672.280000000 -14594 2223005.070000000 -4532 2215711.020000000 -7586 2210562.510000000 -11870 2205182.820000000 -18487 2203653.600000000 -9179 2202720.520000000 -16500 2201185.310000000 -3679 2200592.700000000 -12803 2198295.000000000 -18056 2196741.900000000 -11396 2195645.640000000 -5087 2194120.720000000 -8067 2192048.640000000 -15357 2191646.580000000 -4491 2189713.500000000 -208 2189046.800000000 -10958 2188766.820000000 -9126 2188410.500000000 -15084 2184327.020000000 -18850 2183309.520000000 -3398 2180250.000000000 -16137 2177318.760000000 -211 2174808.960000000 -18422 2174381.000000000 -15840 2173510.400000000 -19553 2173079.770000000 -8221 2169992.160000000 -17000 2169611.160000000 -6755 2168505.150000000 -10817 2167710.680000000 -8327 2167650.600000000 -543 2167368.000000000 -4553 2163371.520000000 -15019 2162288.000000000 -334 2162178.480000000 -8516 2161479.040000000 -11349 2158941.880000000 -3902 2157027.860000000 -14731 2155302.240000000 -326 2153380.080000000 -11403 2151242.300000000 -11657 2150446.080000000 -9496 2149219.010000000 -8110 2149120.130000000 -5153 2148527.250000000 -884 2148324.980000000 -8637 2146185.100000000 -2364 2145790.720000000 -12386 2145001.470000000 -10133 2144903.960000000 -9895 2143324.800000000 -13755 2142539.400000000 -4327 2138501.400000000 -3369 2137408.760000000 -5815 2136985.000000000 -19357 2132657.280000000 -2675 2124158.720000000 -17869 2123991.720000000 -11702 2122132.990000000 -17257 2117850.640000000 -9952 2116686.320000000 -3881 2111457.150000000 -10951 2111185.580000000 -2128 2109702.300000000 -6699 2106578.400000000 -3155 2103636.640000000 -16649 2101956.200000000 -15257 2100297.750000000 -9978 2099566.560000000 -16810 2098301.440000000 -10653 2093388.700000000 -10476 2092766.480000000 -10883 2087495.280000000 -9704 2086967.610000000 -1119 2085182.840000000 -19139 2079788.340000000 -2144 2078391.140000000 -9135 2076377.800000000 -18548 2075584.320000000 -10545 2075230.350000000 -6220 2074341.720000000 -8616 2072887.650000000 -5230 2072161.740000000 -13916 2070504.720000000 -4299 2069922.960000000 -894 2069688.160000000 -17847 2063367.040000000 -18879 2061902.250000000 -13036 2061600.170000000 -10606 2060492.400000000 -9454 2060016.480000000 -118 2059808.860000000 -9601 2059715.760000000 -13769 2057668.080000000 -1987 2057289.270000000 -13863 2055368.000000000 -13562 2054754.240000000 -1840 2054183.920000000 -17995 2053221.900000000 -17389 2051128.200000000 -15168 2045987.490000000 -2139 2045365.400000000 -4024 2044243.100000000 -8964 2041648.850000000 -181 2040167.040000000 -7628 2039548.920000000 -3 2038846.090000000 -15553 2036958.910000000 -11355 2035405.600000000 -13006 2034991.200000000 -3091 2031393.510000000 -1281 2030628.480000000 -1408 2028621.660000000 -18211 2024538.670000000 -2287 2020754.320000000 -6228 2019198.820000000 -4362 2018495.250000000 -10873 2013280.320000000 -7383 2009581.920000000 -1386 2006544.260000000 -9820 2005815.760000000 -18134 2003409.730000000 -15727 2000654.500000000 -157 2000148.160000000 -19571 1999891.110000000 -17728 1997944.400000000 -5278 1996644.210000000 -17737 1994653.760000000 -10220 1989890.980000000 -1397 1984509.220000000 -6195 1983928.260000000 -4270 1983726.950000000 -16965 1983286.250000000 -1683 1980638.640000000 -13086 1978609.400000000 -7124 1974039.380000000 -5211 1973843.760000000 -6794 1973149.470000000 -257 1973035.440000000 -6995 1968281.550000000 -8447 1967292.700000000 -15873 1967257.890000000 -12862 1964014.130000000 -8295 1961467.080000000 -931 1958825.220000000 -6876 1957359.480000000 -1932 1954592.400000000 -1061 1952688.060000000 -18108 1951143.670000000 -5138 1950861.000000000 -12598 1950211.610000000 -10829 1943924.620000000 -11950 1941211.000000000 -12076 1939323.960000000 -2176 1938691.370000000 -6616 1937401.880000000 -5893 1934358.580000000 -976 1933066.800000000 -13173 1932557.520000000 -14947 1929229.980000000 -16857 1928814.800000000 -13403 1928702.880000000 -4819 1926969.680000000 -13127 1926929.830000000 -6871 1926787.680000000 -15465 1925145.090000000 -1131 1920005.500000000 -11845 1913576.400000000 -8364 1909122.200000000 -16588 1904272.370000000 -6759 1903906.290000000 -11586 1901895.650000000 -8145 1901787.660000000 -17333 1897297.200000000 -13290 1890633.750000000 -6499 1887621.000000000 -4881 1887535.920000000 -7147 1886710.200000000 -3883 1886567.780000000 -18911 1885597.120000000 -11336 1883573.600000000 -8653 1883275.760000000 -19476 1881492.480000000 -14799 1880543.400000000 -14491 1879219.920000000 -11815 1877434.340000000 -3173 1874302.100000000 -7161 1873023.450000000 -14631 1873015.300000000 -4247 1869912.960000000 -3568 1865824.400000000 -1500 1865451.030000000 -11833 1863665.230000000 -495 1860771.300000000 -6776 1855589.170000000 -11374 1855221.120000000 -5637 1853782.170000000 -3597 1852826.800000000 -981 1852083.600000000 -16076 1850349.690000000 -17597 1845420.950000000 -19609 1843185.480000000 -10997 1843072.020000000 -3403 1842975.000000000 -897 1842845.100000000 -16697 1840630.680000000 -17644 1840597.800000000 -6485 1838812.020000000 -5492 1836202.880000000 -12038 1835075.060000000 -9325 1832634.840000000 -10637 1832347.440000000 -11318 1830158.390000000 -4357 1828730.000000000 -18553 1826335.200000000 -12623 1825950.850000000 -961 1825869.600000000 -1677 1821816.900000000 -8211 1820432.520000000 -19719 1819333.550000000 -19663 1819074.350000000 -16296 1818353.770000000 -16527 1817834.420000000 -4964 1815400.020000000 -1769 1812929.200000000 -13126 1808799.960000000 -7854 1807608.060000000 -18380 1803641.220000000 -6584 1802346.980000000 -7665 1801765.350000000 -16553 1796146.780000000 -17761 1795095.720000000 -11179 1794890.300000000 -15171 1794148.720000000 -3018 1793183.880000000 -15741 1788612.000000000 -5331 1783901.350000000 -9860 1775071.260000000 -7984 1774302.750000000 -15354 1774270.770000000 -17884 1774212.440000000 -16257 1771869.710000000 -10696 1768645.200000000 -2104 1767902.640000000 -14465 1764946.400000000 -10089 1764692.320000000 -6719 1762699.540000000 -3648 1760594.420000000 -7241 1759913.590000000 -11122 1757430.040000000 -17019 1752560.650000000 -13877 1744271.100000000 -15325 1743826.260000000 -17860 1739870.440000000 -2236 1739795.800000000 -4436 1738760.320000000 -7701 1738670.400000000 -8147 1736855.160000000 -6676 1736341.440000000 -19505 1735413.430000000 -9885 1731366.260000000 -2112 1725934.080000000 -5330 1722196.980000000 -3561 1720377.960000000 -10104 1714419.160000000 -16362 1712457.380000000 -15573 1712365.440000000 -15006 1711381.350000000 -14629 1709942.050000000 -9612 1709528.380000000 -19910 1709211.150000000 -13145 1708907.460000000 -11494 1707973.680000000 -15895 1706999.450000000 -8239 1705479.100000000 -2403 1705331.100000000 -19436 1702706.000000000 -3476 1702335.800000000 -6828 1702292.080000000 -771 1701589.500000000 -8448 1700312.440000000 -3755 1699047.030000000 -13895 1698679.030000000 -9785 1698056.370000000 -6180 1695571.530000000 -532 1694356.150000000 -6741 1692552.420000000 -19964 1692367.640000000 -3747 1691244.600000000 -3253 1690719.420000000 -16119 1688339.250000000 -7113 1681911.000000000 -12368 1681219.800000000 -16378 1679705.600000000 -1393 1675545.350000000 -11119 1675453.440000000 -4469 1674023.490000000 -6955 1672618.900000000 -11579 1672345.320000000 -19898 1671781.700000000 -15351 1659204.300000000 -6133 1658215.460000000 -9110 1658054.680000000 -2979 1656016.740000000 -18764 1653708.480000000 -8995 1653627.580000000 -13096 1651408.670000000 -15062 1650548.020000000 -7924 1650202.400000000 -10076 1647970.240000000 -15859 1646036.280000000 -17932 1642640.660000000 -19694 1642089.500000000 -13827 1642001.310000000 -17963 1639689.000000000 -10698 1635848.260000000 -18003 1633530.780000000 -8416 1633366.770000000 -476 1631154.060000000 -2806 1630782.800000000 -12129 1628615.470000000 -11215 1626624.700000000 -14061 1624933.440000000 -5956 1623586.100000000 -9043 1622670.400000000 -13287 1621980.360000000 -11410 1621420.900000000 -13990 1621268.200000000 -12952 1619215.180000000 -15181 1619088.680000000 -9784 1618120.530000000 -10733 1616168.880000000 -16054 1614531.230000000 -5864 1614397.830000000 -1875 1611927.000000000 -17381 1611664.800000000 -14562 1607467.920000000 -575 1605941.730000000 -2005 1605591.720000000 -4332 1605448.830000000 -4653 1602596.300000000 -15403 1601830.440000000 -17430 1599681.420000000 -4798 1593630.500000000 -12991 1593321.520000000 -15653 1593138.660000000 -10066 1593049.060000000 -8892 1592100.900000000 -6708 1590159.120000000 -9825 1589403.920000000 -8271 1588475.410000000 -17084 1584280.880000000 -4003 1583631.000000000 -869 1582643.160000000 -16400 1582313.200000000 -19088 1581708.560000000 -6581 1581346.800000000 -9481 1581048.600000000 -6092 1580846.490000000 -3624 1578777.300000000 -6503 1578507.780000000 -14557 1578280.960000000 -2428 1577543.920000000 -15513 1573560.210000000 -4641 1573363.540000000 -10152 1570213.600000000 -5932 1566902.520000000 -7482 1561323.500000000 -13745 1558358.340000000 -2251 1558274.700000000 -9845 1558068.120000000 -7603 1557388.200000000 -1809 1553837.200000000 -18128 1547643.360000000 -8086 1543199.040000000 -14948 1541721.570000000 -16725 1540948.500000000 -2999 1540317.660000000 -8861 1540008.470000000 -1964 1538815.250000000 -19374 1537884.780000000 -15428 1535994.360000000 -7449 1534782.480000000 -16884 1534509.160000000 -10271 1534397.340000000 -11782 1529963.220000000 -8184 1529750.700000000 -4560 1527433.240000000 -4616 1525374.460000000 -3814 1524077.040000000 -17265 1523932.080000000 -16520 1522906.280000000 -10475 1518705.060000000 -5094 1517317.830000000 -8626 1515142.070000000 -19895 1512286.680000000 -19933 1506235.360000000 -6854 1505626.000000000 -13995 1505562.180000000 -7102 1504945.670000000 -9079 1501237.200000000 -18329 1500146.900000000 -3742 1496990.770000000 -12395 1496904.430000000 -12214 1496489.400000000 -12298 1495554.300000000 -4978 1495389.500000000 -2927 1494280.100000000 -2119 1494151.140000000 -15143 1492039.750000000 -14548 1487406.600000000 -840 1486128.980000000 -5902 1486097.280000000 -10614 1482144.720000000 -5895 1481356.800000000 -15958 1480951.600000000 -11408 1479948.960000000 -8407 1474236.000000000 -6243 1471007.850000000 -10389 1469004.460000000 -13871 1468938.640000000 -19811 1464597.090000000 -10495 1464290.490000000 -4389 1463010.830000000 -1311 1461703.360000000 -17874 1459408.880000000 -6597 1458761.870000000 -19211 1456741.630000000 -12879 1456178.240000000 -8840 1455731.460000000 -14755 1454890.600000000 -16957 1454465.960000000 -9257 1454388.760000000 -5193 1454011.320000000 -6884 1452474.600000000 -19948 1452024.000000000 -15076 1448395.000000000 -16016 1447557.450000000 -11693 1445839.680000000 -6975 1440516.960000000 -4290 1439768.460000000 -18900 1438722.100000000 -14383 1438477.920000000 -15098 1435941.780000000 -9322 1435282.800000000 -458 1433040.450000000 -10042 1432906.350000000 -5052 1431900.900000000 -6600 1431116.550000000 -3630 1428665.040000000 -9636 1428193.840000000 -16511 1427308.740000000 -4045 1427248.350000000 -19562 1426348.820000000 -8814 1425690.090000000 -2616 1425178.040000000 -4587 1425109.400000000 -148 1424237.370000000 -2712 1423780.260000000 -10863 1423386.160000000 -16096 1421942.090000000 -18936 1421938.650000000 -18327 1419872.920000000 -11620 1419050.100000000 -3740 1418609.850000000 -3457 1418603.500000000 -1185 1417637.470000000 -8178 1417357.260000000 -17791 1413293.130000000 -13608 1411323.120000000 -17849 1409613.500000000 -6814 1406228.400000000 -14022 1406138.040000000 -14231 1403771.520000000 -19546 1402854.600000000 -19619 1402389.160000000 -5609 1402302.540000000 -5342 1401567.590000000 -3084 1401096.100000000 -5708 1400334.900000000 -17998 1399862.450000000 -19850 1397630.330000000 -14004 1395443.100000000 -13071 1394653.240000000 -2797 1393747.580000000 -2866 1392947.250000000 -19809 1389067.680000000 -13600 1380865.800000000 -13614 1380654.360000000 -5884 1380319.740000000 -9404 1378623.660000000 -10656 1376954.320000000 -12324 1376502.400000000 -7325 1375030.430000000 -13295 1373987.340000000 -11864 1373555.680000000 -6987 1373481.510000000 -8386 1371854.410000000 -10916 1370374.320000000 -12867 1369058.110000000 -14668 1369040.340000000 -13383 1367342.300000000 -18572 1366953.960000000 -1152 1366861.380000000 -6015 1366452.180000000 -3344 1366185.150000000 -7889 1365521.920000000 -13345 1364088.880000000 -6276 1363421.620000000 -8069 1361824.200000000 -17509 1360892.490000000 -15137 1358678.070000000 -17163 1357391.520000000 -4704 1356692.400000000 -8609 1356578.190000000 -12644 1356088.140000000 -17141 1356022.380000000 -11805 1354826.780000000 -6386 1354187.220000000 -3004 1352173.440000000 -8634 1350211.800000000 -4399 1349881.200000000 -10362 1349411.340000000 -1572 1348835.200000000 -7359 1348224.100000000 -11884 1346696.820000000 -11671 1346424.150000000 -5350 1346359.280000000 -3119 1345996.480000000 -5307 1345356.000000000 -16117 1345045.120000000 -8715 1342665.720000000 -5398 1341179.280000000 -7627 1338820.560000000 -8457 1337714.680000000 -4958 1334732.710000000 -84 1334146.710000000 -6932 1333235.360000000 -757 1332921.070000000 -4076 1332441.000000000 -1751 1329112.320000000 -15701 1327052.160000000 -4119 1326549.900000000 -1562 1325604.280000000 -8741 1325517.600000000 -1135 1325422.710000000 -1002 1323418.650000000 -5832 1323085.710000000 -5368 1322793.960000000 -5382 1322628.840000000 -5616 1319082.260000000 -2832 1318691.950000000 -3895 1317858.440000000 -8629 1317756.510000000 -5709 1317058.680000000 -18383 1316451.050000000 -15797 1314806.640000000 -1900 1313660.400000000 -13882 1310455.860000000 -6785 1309877.800000000 -14855 1309280.760000000 -7761 1308602.240000000 -14268 1306810.400000000 -6257 1306056.960000000 -19002 1305509.520000000 -5095 1303729.020000000 -10320 1301657.580000000 -7826 1299561.680000000 -13359 1298717.140000000 -7436 1298127.360000000 -5644 1295055.770000000 -11327 1290526.410000000 -5277 1289329.650000000 -15932 1286235.840000000 -14322 1284809.360000000 -144 1284270.120000000 -3043 1281162.790000000 -16788 1280955.340000000 -17136 1280443.120000000 -12560 1279117.950000000 -13833 1278834.750000000 -5414 1277893.260000000 -12582 1277592.320000000 -4644 1277535.000000000 -14032 1277077.880000000 -18325 1271719.680000000 -7072 1271228.480000000 -16868 1267469.420000000 -8137 1267425.810000000 -5976 1266206.850000000 -14125 1265569.050000000 -13299 1265287.550000000 -18376 1264249.300000000 -6157 1261759.920000000 -5002 1261669.640000000 -13368 1260918.600000000 -15589 1260059.760000000 -2149 1258981.440000000 -9639 1256283.380000000 -11689 1256027.920000000 -9083 1245924.240000000 -16231 1242625.650000000 -5084 1242385.280000000 -11634 1240760.180000000 -15617 1239731.250000000 -9865 1237181.620000000 -14212 1236365.520000000 -10325 1235223.360000000 -19582 1235105.760000000 -740 1234746.810000000 -19231 1233623.100000000 -16840 1233063.850000000 -5703 1231744.330000000 -5761 1229435.200000000 -15630 1226611.620000000 -10408 1224698.400000000 -9177 1221942.510000000 -13389 1221666.750000000 -6104 1221577.920000000 -9673 1218826.640000000 -2707 1217124.480000000 -18672 1214208.800000000 -5112 1209590.200000000 -6264 1208318.500000000 -18496 1207881.750000000 -10971 1207183.520000000 -19059 1206729.900000000 -431 1205938.440000000 -3821 1201192.750000000 -826 1200454.620000000 -3317 1200440.900000000 -19689 1198899.520000000 -19641 1198797.990000000 -6379 1197195.500000000 -814 1194417.400000000 -18643 1194000.780000000 -11865 1193965.760000000 -12393 1193896.800000000 -9218 1193660.580000000 -8674 1191881.320000000 -8582 1191804.020000000 -13084 1191508.000000000 -18844 1190239.960000000 -16061 1189935.000000000 -6134 1185550.800000000 -8628 1183245.600000000 -8884 1181547.480000000 -7697 1181032.500000000 -9044 1180922.600000000 -13257 1180158.570000000 -8066 1178808.120000000 -5876 1177376.800000000 -14694 1177059.310000000 -16062 1175391.000000000 -9104 1175178.900000000 -11600 1175091.060000000 -10337 1172684.920000000 -19188 1172349.780000000 -8833 1171372.930000000 -6895 1170602.070000000 -14100 1168878.400000000 -13538 1168554.280000000 -3408 1166645.160000000 -1860 1165673.680000000 -13436 1164278.700000000 -19325 1162733.700000000 -7403 1161982.000000000 -4882 1161404.810000000 -13105 1161320.580000000 -17880 1161256.020000000 -19284 1160927.600000000 -13476 1159035.150000000 -18913 1158208.300000000 -18523 1158135.000000000 -12508 1157538.450000000 -9090 1156362.640000000 -17653 1154338.080000000 -3926 1152652.520000000 -10183 1148324.570000000 -7556 1146268.140000000 -16436 1142656.470000000 -4741 1141614.000000000 -15651 1141497.930000000 -3183 1140081.360000000 -9532 1139902.500000000 -16403 1139306.370000000 -2368 1137421.160000000 -3889 1136395.500000000 -2885 1135838.140000000 -7851 1135110.760000000 -16234 1135017.240000000 -12746 1134531.040000000 -2647 1132941.120000000 -5373 1132158.010000000 -10340 1132004.240000000 -8873 1131949.280000000 -1132 1131338.880000000 -15594 1131328.620000000 -4376 1130282.200000000 -240 1126682.480000000 -2231 1124447.150000000 -929 1121383.920000000 -11599 1119307.270000000 -3765 1119093.500000000 -17635 1118420.160000000 -7119 1118285.080000000 -15121 1117715.340000000 -11858 1116963.540000000 -16963 1116929.450000000 -16356 1113648.980000000 -6924 1112198.400000000 -16223 1111257.000000000 -18091 1110043.020000000 -12628 1108954.800000000 -16043 1108831.050000000 -9402 1108290.480000000 -708 1107084.000000000 -4078 1105993.960000000 -17593 1104713.400000000 -12776 1104362.590000000 -7583 1102813.530000000 -14619 1102675.800000000 -8842 1100110.260000000 -4196 1099726.550000000 -2019 1098178.640000000 -6863 1097246.360000000 -6489 1096503.070000000 -2459 1094813.040000000 -11964 1094485.020000000 -3236 1093969.800000000 -17647 1093809.150000000 -17648 1093114.620000000 -119 1092687.480000000 -9626 1092080.000000000 -9124 1091569.680000000 -13175 1089851.760000000 -2532 1088706.350000000 -16083 1088295.390000000 -8874 1086011.340000000 -12872 1082970.300000000 -19821 1082520.840000000 -4800 1080389.700000000 -18696 1079685.360000000 -19545 1079184.330000000 -13120 1077742.280000000 -10588 1076203.830000000 -17696 1075092.720000000 -14651 1073222.230000000 -903 1071146.760000000 -5858 1070259.480000000 -8302 1069504.800000000 -18728 1069225.510000000 -18026 1068569.000000000 -19383 1066907.580000000 -18690 1065930.900000000 -5924 1065143.120000000 -4880 1065011.750000000 -12439 1064381.190000000 -16529 1062371.700000000 -19653 1057683.560000000 -3136 1056810.440000000 -18932 1056193.650000000 -2124 1054160.520000000 -16851 1052646.840000000 -10123 1051624.000000000 -5618 1048447.930000000 -19851 1045187.850000000 -16278 1044808.380000000 -11479 1044276.220000000 -13263 1042046.200000000 -6041 1041123.380000000 -7193 1040455.320000000 -19408 1039430.010000000 -11260 1036828.520000000 -5179 1035633.440000000 -1331 1034398.000000000 -7706 1034249.400000000 -8436 1033549.350000000 -1801 1031886.000000000 -4170 1031642.900000000 -11827 1031139.390000000 -17114 1027985.880000000 -18278 1026583.110000000 -1995 1025165.680000000 -7667 1022980.150000000 -6559 1021635.450000000 -17488 1021612.130000000 -16059 1019781.190000000 -7633 1018782.570000000 -10032 1016809.500000000 -2899 1016438.760000000 -14628 1016033.200000000 -10126 1015846.780000000 -3884 1014413.500000000 -16913 1013604.400000000 -18644 1010288.100000000 -19870 1007919.360000000 -18564 1007416.200000000 -10179 1004920.000000000 -883 1004650.680000000 -3627 1004461.040000000 +12098 16227681.21 +5134 15709338.52 +13334 15023662.41 +17052 14351644.20 +3452 14070870.14 +12552 13332469.18 +1084 13170428.29 +5797 13038622.72 +12633 12892561.61 +403 12856217.34 +1833 12024581.72 +2084 11502875.36 +17349 11354213.05 +18427 11282385.24 +2860 11262529.95 +17852 10934711.93 +9871 10889253.68 +12231 10841131.39 +6366 10759786.81 +12146 10257362.66 +5043 10226395.88 +12969 10125777.93 +1504 10004397.08 +14327 9981697.08 +134 9965150.66 +6860 9805871.26 +10624 9776138.40 +15819 9775705.31 +3293 9674928.12 +19865 9653766.83 +8870 9648981.87 +15778 9636332.82 +12360 9635023.92 +14389 9475588.34 +3257 9451029.24 +9476 9435207.28 +19629 9391236.40 +7179 9386222.25 +15723 9383900.80 +4054 9313810.02 +2380 9307751.22 +19084 9302916.80 +4703 9280804.80 +18791 9267017.97 +19994 9235972.92 +9149 9121803.90 +15118 9120819.50 +6116 9079369.20 +7052 9077468.92 +14147 9069193.78 +7305 9035228.53 +9130 9024379.25 +16698 8991337.95 +1553 8977226.10 +16777 8961355.62 +1402 8953779.12 +18963 8934063.40 +8358 8930611.48 +17547 8860117.00 +5128 8844222.75 +17063 8840649.60 +15490 8833581.40 +14761 8817240.56 +19601 8791341.02 +16160 8740262.76 +13597 8702669.82 +13653 8693170.16 +16383 8691505.92 +325 8667741.28 +8879 8667584.38 +10564 8667098.22 +17429 8661827.90 +17403 8643350.30 +18294 8616583.43 +4181 8592684.66 +13008 8567480.64 +13211 8537000.01 +1884 8532644.34 +11101 8530945.32 +11562 8528028.57 +15878 8523591.84 +834 8522135.27 +2423 8517902.85 +15383 8513433.11 +18119 8507611.80 +7389 8506099.20 +5016 8489784.15 +17473 8444766.24 +6669 8428618.46 +384 8418472.27 +12052 8411519.28 +17562 8409022.83 +8128 8379149.47 +13813 8374830.84 +12800 8318626.78 +10887 8315019.36 +1644 8285453.08 +16638 8274568.00 +1394 8255140.60 +7219 8254985.30 +13358 8253829.80 +5562 8252365.16 +14861 8242296.15 +15416 8196621.53 +1963 8192206.61 +2841 8148678.45 +6635 8122918.28 +3609 8099812.05 +6372 8093695.06 +5231 8091704.18 +8755 8085017.03 +4071 8083755.18 +4977 8058501.72 +11873 8057540.30 +12127 8051142.94 +2401 8049525.94 +15964 8037547.75 +10129 8030855.04 +7605 8028622.42 +9327 8022291.30 +11814 7983589.59 +4646 7981660.60 +6032 7981511.28 +1076 7977074.75 +4043 7971929.90 +8932 7967222.19 +13705 7953344.52 +16855 7923819.00 +3234 7920022.00 +17752 7901362.77 +2097 7892993.17 +18599 7890774.40 +19596 7874564.59 +11392 7861172.48 +18121 7857581.60 +17452 7838465.36 +6076 7821316.80 +15134 7804760.25 +8641 7802917.15 +2134 7800185.43 +16352 7797112.00 +19312 7775952.24 +2895 7759327.23 +12860 7758135.21 +153 7755681.28 +15089 7735438.26 +14797 7725353.16 +15946 7722773.88 +10919 7722425.36 +9867 7721597.78 +11881 7713136.42 +16552 7708518.15 +6925 7703999.68 +12147 7703826.98 +8923 7702690.28 +16116 7697970.84 +10661 7673830.20 +17094 7665368.16 +18648 7650862.02 +12172 7641326.40 +15123 7633032.50 +4993 7600570.80 +17162 7592062.56 +13506 7580809.83 +3436 7575616.33 +6271 7559793.93 +13314 7555156.63 +17242 7550949.50 +2753 7549574.06 +7391 7543159.68 +7418 7541449.65 +116 7520874.24 +12436 7520234.54 +1181 7494798.28 +12963 7491248.10 +213 7479470.28 +7114 7477681.20 +18521 7476478.30 +8973 7458603.67 +4202 7454095.74 +12009 7442105.40 +10609 7429346.40 +5622 7424142.66 +5143 7422760.28 +898 7414133.80 +12257 7408190.63 +6740 7400350.35 +1146 7394394.48 +5485 7378181.94 +8437 7376353.59 +6531 7362366.78 +16463 7362106.80 +10412 7359552.42 +12857 7340801.58 +12354 7332343.20 +7616 7320032.37 +3426 7312340.41 +8622 7307266.33 +6818 7304782.10 +3705 7299380.10 +12733 7298398.17 +1574 7293446.04 +10554 7289933.72 +9363 7284647.70 +4611 7282115.00 +7857 7266948.92 +9616 7265005.30 +15860 7254466.66 +15554 7247592.51 +3422 7247586.45 +9134 7236404.67 +17408 7220083.60 +15853 7219984.90 +9266 7218517.11 +1881 7208346.45 +10148 7205335.83 +8860 7202401.44 +8373 7189039.50 +10859 7188990.69 +12670 7188177.22 +2553 7180006.02 +19346 7176797.25 +1681 7160169.84 +15225 7158861.32 +1052 7158586.00 +77 7155531.10 +7231 7155250.38 +12622 7137408.42 +9814 7105363.14 +8695 7103187.00 +13174 7099182.53 +6179 7095134.05 +6451 7082495.36 +19860 7073206.83 +9307 7059973.68 +7819 7055963.04 +10556 7053491.07 +9366 7048690.74 +12124 7040021.31 +11476 7037906.76 +19245 7034045.24 +7562 7030275.79 +12290 7020372.06 +18118 7003396.80 +1253 7001569.62 +8662 6999834.27 +1779 6997385.73 +15386 6996871.79 +147 6989079.88 +9562 6983076.90 +3958 6969833.22 +7211 6966606.67 +12842 6923277.66 +4368 6918783.95 +11131 6918656.78 +4628 6894893.82 +5879 6881367.14 +16586 6865931.80 +32 6852925.59 +12119 6842773.70 +1371 6831137.52 +6136 6827917.01 +13857 6824240.60 +1074 6821747.88 +1863 6821522.19 +14597 6817385.66 +9271 6783068.88 +6389 6781075.68 +11703 6776538.36 +13701 6768880.56 +14880 6763788.24 +18428 6763670.54 +1006 6762065.94 +4927 6756765.21 +11659 6755246.68 +6815 6738928.35 +14367 6733857.20 +10703 6730936.46 +3150 6727920.40 +9963 6725919.35 +10438 6710153.62 +4745 6682153.67 +15297 6681711.28 +10848 6678666.25 +11749 6677895.73 +16739 6675549.12 +14915 6672248.66 +19841 6669191.20 +628 6666892.90 +1650 6657178.32 +7069 6648672.24 +7108 6646445.96 +8887 6641655.90 +18735 6636457.56 +3711 6632665.38 +2325 6630475.92 +6484 6622965.20 +2643 6617304.98 +7895 6615061.26 +12840 6604848.54 +4314 6600905.71 +19077 6591091.97 +17476 6576029.00 +7039 6559838.90 +8106 6558223.02 +2391 6557120.08 +7717 6547706.88 +12476 6546988.15 +9155 6540793.60 +360 6530297.41 +6383 6529336.02 +4830 6518998.92 +12600 6511549.46 +13740 6508057.92 +4678 6507847.98 +17815 6502284.76 +7329 6490811.95 +13884 6490063.10 +5147 6487069.00 +16548 6482024.50 +14144 6476413.40 +10181 6474984.88 +11031 6463308.02 +19958 6461506.38 +6043 6458177.64 +10060 6455476.89 +9144 6454042.05 +7043 6448019.98 +17346 6444307.52 +13963 6442014.48 +7111 6441947.07 +14140 6439955.54 +2327 6438977.28 +14812 6437152.54 +16755 6430895.36 +14840 6430549.14 +14134 6422079.26 +12655 6405496.79 +1518 6390148.22 +9888 6385033.02 +7387 6384005.18 +5393 6381083.04 +11057 6375974.22 +16818 6368828.80 +4576 6364925.71 +19644 6350000.33 +139 6336065.55 +11735 6334305.90 +10108 6332055.99 +15628 6329801.34 +4349 6314949.68 +7048 6313868.55 +17119 6298935.38 +18597 6291416.31 +2488 6286372.05 +2296 6275519.25 +4343 6272834.20 +9958 6267714.61 +2147 6267475.32 +9368 6254578.92 +13134 6252432.76 +10119 6251456.25 +2093 6249342.36 +2392 6237476.56 +17231 6233509.18 +3773 6226654.68 +9839 6214044.10 +19349 6213969.30 +7869 6212751.00 +8158 6210279.24 +13179 6205773.65 +2349 6205589.26 +9052 6200729.70 +1744 6189967.36 +597 6183103.47 +16721 6176606.60 +7498 6176277.25 +15157 6176222.50 +17524 6171107.36 +7922 6170906.07 +6615 6170738.42 +2706 6165991.65 +16432 6162740.68 +16473 6162427.96 +12825 6161595.60 +18813 6154678.55 +1030 6146500.28 +2571 6145772.43 +7707 6144754.71 +10327 6137612.00 +4710 6132346.56 +3649 6130602.63 +4893 6128461.24 +12844 6128191.24 +8794 6122690.28 +1157 6117749.22 +10895 6112017.68 +16166 6108250.98 +11920 6107122.56 +5621 6102123.56 +6141 6093826.56 +14076 6086671.08 +15884 6080485.59 +4814 6080337.96 +5814 6079842.96 +1134 6078685.20 +19048 6071813.28 +303 6070601.35 +15211 6065678.06 +1020 6054168.00 +11292 6052522.83 +7134 6049865.25 +14119 6049619.55 +2389 6042429.12 +5926 6034269.52 +8553 6030922.95 +18814 6023255.17 +12702 6023190.99 +2644 6020931.33 +19628 6010704.00 +18112 6008707.52 +13860 6008170.29 +1456 6005092.14 +1633 6002068.85 +2301 6000302.29 +10740 5999809.05 +2630 5997008.43 +8818 5992296.90 +10043 5990594.02 +653 5987942.83 +6829 5985990.66 +15179 5977727.52 +9663 5973523.92 +5863 5973328.92 +3628 5966340.09 +7618 5960155.86 +2588 5952648.56 +4865 5949383.40 +4233 5944699.60 +13390 5944104.69 +16321 5942714.70 +9653 5941308.50 +18884 5925548.24 +4394 5920927.14 +19774 5916723.12 +1257 5914052.36 +2963 5911917.77 +17157 5899573.02 +383 5884693.36 +11709 5884134.48 +18135 5871431.61 +13169 5869736.88 +2932 5868995.80 +2888 5863229.30 +6820 5853106.80 +18622 5850951.60 +9066 5846052.72 +19840 5832251.40 +6325 5827298.43 +14092 5823711.72 +11532 5823277.12 +18043 5815248.70 +3932 5809134.60 +10364 5808371.40 +1179 5808303.24 +11441 5799378.33 +15767 5798263.26 +14063 5797204.00 +11383 5793893.12 +10065 5781764.45 +17785 5766838.76 +18349 5761765.77 +14824 5760339.78 +14699 5759490.18 +11628 5755137.20 +4403 5752376.64 +13090 5751949.25 +15332 5744606.28 +17573 5744384.15 +12895 5741007.02 +13577 5739209.94 +16594 5732523.64 +8497 5727509.84 +2872 5724068.72 +16181 5721788.16 +6639 5712041.22 +13095 5708798.25 +4535 5693341.20 +10956 5692585.50 +19755 5686913.37 +12995 5682200.41 +13157 5681454.94 +1887 5681341.24 +18472 5680356.00 +19814 5679360.00 +18583 5669360.52 +3894 5664385.49 +1740 5659552.04 +62 5659470.16 +16532 5653779.46 +995 5648973.45 +7486 5646851.87 +19007 5642710.93 +13060 5642024.64 +12371 5635710.45 +2280 5634077.40 +3115 5631352.32 +11107 5631252.46 +5873 5629125.75 +14743 5628732.80 +2224 5624746.62 +2653 5623859.99 +17622 5623057.44 +14700 5615344.00 +14740 5613929.39 +6403 5611924.51 +6896 5609169.88 +10384 5607337.52 +16433 5605707.03 +5898 5604937.76 +4789 5600488.36 +8132 5593107.50 +3838 5592313.20 +13631 5586424.80 +11233 5585247.01 +849 5583516.45 +14653 5581550.45 +14788 5580433.00 +18181 5578562.88 +19815 5577102.62 +5584 5576692.20 +5385 5576420.19 +13780 5569028.61 +9342 5566783.71 +19056 5566524.12 +8189 5565694.74 +13808 5560721.96 +10635 5560058.55 +8304 5550784.41 +14257 5549164.06 +8999 5542100.10 +19134 5539312.56 +8360 5538031.10 +6397 5536651.92 +2597 5525317.76 +8631 5515909.38 +16729 5512663.65 +11861 5511785.93 +16853 5511689.91 +6341 5502790.08 +1312 5496649.40 +5566 5495885.87 +12519 5490649.97 +19032 5488105.02 +8231 5479312.50 +3026 5466732.32 +6388 5466168.80 +15349 5464571.52 +18985 5463897.13 +19848 5454266.40 +17378 5453284.74 +4000 5448690.39 +3710 5445822.53 +13181 5439774.06 +6420 5438325.32 +3644 5437772.14 +1117 5437024.97 +13027 5436968.46 +12884 5432632.34 +12781 5429161.08 +10084 5428231.62 +5640 5423318.58 +16208 5422901.40 +753 5416899.74 +4120 5413822.46 +12348 5412061.71 +1586 5411198.55 +2503 5411125.80 +1381 5397790.10 +19673 5397746.77 +19389 5394426.40 +15532 5386361.38 +3562 5380335.08 +19114 5375133.00 +3204 5372215.46 +6653 5365178.00 +3553 5363690.29 +12687 5361682.12 +3279 5357505.65 +9840 5350112.18 +8258 5347064.67 +11387 5345284.00 +15210 5341117.92 +15092 5340896.80 +6052 5339762.18 +14043 5339587.68 +6284 5336153.00 +6297 5332357.74 +16254 5326607.83 +18436 5326198.50 +14436 5325517.14 +10368 5319983.08 +6349 5317943.40 +19984 5317620.00 +19080 5310678.60 +1403 5306771.32 +5995 5305056.12 +13214 5299420.45 +19293 5297054.40 +7130 5289761.48 +9809 5286784.80 +9273 5277924.77 +16786 5260035.00 +11032 5256354.35 +17719 5246322.20 +3400 5244203.68 +8278 5243718.48 +7873 5241168.80 +9357 5239695.77 +2239 5235767.83 +18695 5223960.35 +19271 5216498.29 +4563 5214673.70 +1896 5210699.79 +15755 5210481.24 +17628 5209882.37 +5747 5206892.24 +18057 5204907.40 +5075 5204412.81 +5325 5190162.75 +17675 5183135.36 +18818 5181295.30 +11064 5180667.57 +12578 5177222.33 +3725 5169183.75 +11112 5161546.00 +4749 5161499.44 +7632 5159180.50 +13119 5151665.51 +5653 5140324.64 +16134 5137482.51 +5671 5136815.55 +18348 5132981.07 +562 5132487.25 +2562 5126763.83 +10304 5125180.00 +7622 5120719.98 +12755 5118213.92 +1814 5100705.58 +11269 5096671.33 +2964 5095253.72 +1616 5091834.00 +9294 5090753.53 +16793 5085330.54 +4999 5081651.75 +17209 5079029.28 +7151 5078937.60 +15522 5072469.60 +3056 5072329.55 +7612 5068322.87 +18453 5063892.92 +18324 5058901.22 +12266 5058186.75 +19394 5056235.73 +1713 5054968.05 +15681 5051569.63 +8274 5043328.00 +18160 5043074.83 +18253 5041572.00 +11840 5040590.04 +1532 5033171.00 +584 5031602.64 +12382 5028901.00 +14814 5022200.07 +19058 5019001.92 +4487 5016640.86 +8482 5015444.25 +18476 5011136.36 +12335 5003581.40 +4455 4997933.31 +14355 4992822.92 +15253 4992642.20 +14069 4983244.20 +17843 4977294.37 +9389 4975885.83 +14435 4971442.19 +13254 4959481.45 +9773 4955887.80 +7615 4952421.54 +6476 4947250.05 +9253 4945159.70 +14932 4934395.48 +13253 4932867.45 +19322 4931525.78 +16945 4931440.61 +731 4930191.93 +6540 4925114.51 +5148 4923048.00 +1934 4921196.90 +15402 4920840.72 +17914 4919607.04 +5416 4916041.92 +16734 4914205.27 +14967 4900262.08 +8706 4894595.58 +136 4891960.92 +19494 4886028.30 +8737 4880640.44 +7653 4879423.64 +4149 4875782.40 +7890 4872424.40 +11142 4871415.42 +10386 4863623.50 +8603 4861814.61 +2680 4861223.74 +4891 4858103.88 +19236 4855097.69 +14251 4854739.86 +18895 4853365.10 +17134 4852041.04 +4932 4843701.45 +10033 4841647.72 +1383 4839144.31 +18721 4837983.36 +8618 4833125.42 +17386 4831545.84 +3790 4830282.36 +1043 4825921.31 +12434 4822597.78 +18385 4819643.40 +6046 4817460.06 +5821 4814423.45 +10836 4814303.24 +6848 4813954.08 +6880 4804600.35 +11249 4800116.82 +11970 4799739.66 +14253 4796521.29 +7782 4793227.13 +75 4790042.88 +7076 4789347.34 +9566 4782531.80 +2137 4767931.74 +2336 4763870.79 +15362 4759043.38 +3284 4755048.76 +12964 4753627.48 +1781 4752835.20 +5454 4748342.98 +12597 4742077.84 +19120 4733459.96 +14884 4731499.44 +825 4730720.28 +14683 4730482.32 +5361 4726113.00 +12179 4725018.10 +1461 4710954.69 +9890 4709658.40 +13369 4705085.39 +11612 4701627.99 +3096 4699414.40 +10383 4697866.47 +11293 4697336.06 +3383 4695825.20 +6231 4694381.72 +7396 4691319.06 +17827 4688797.44 +15856 4683368.21 +8253 4678560.86 +12327 4677984.74 +4395 4676829.82 +4232 4676646.40 +14260 4670522.80 +15288 4669273.99 +17526 4668545.64 +9884 4662693.84 +2118 4660352.78 +4524 4653956.60 +19090 4650872.94 +3928 4649359.44 +14325 4647762.17 +15476 4643469.04 +4179 4639931.76 +14408 4639631.28 +19424 4634817.44 +3334 4633102.50 +9477 4628073.51 +11803 4625396.80 +14805 4618452.18 +463 4616307.28 +16628 4607490.96 +3116 4604463.10 +19962 4602949.47 +12859 4602870.55 +12063 4600708.45 +5648 4592273.25 +8556 4590726.86 +15281 4589425.41 +9414 4587426.90 +13951 4586281.25 +19328 4582624.82 +15963 4579705.50 +10773 4573276.20 +14179 4568816.00 +1895 4563988.16 +6408 4561496.39 +5958 4554000.00 +3653 4548134.40 +11218 4546237.92 +19327 4543987.77 +9572 4535941.16 +14556 4531464.75 +2475 4529761.50 +9631 4529261.56 +1901 4528592.55 +86 4528475.38 +9586 4527146.22 +17361 4519098.87 +8112 4514949.45 +13468 4499728.20 +18239 4497633.64 +10215 4494553.60 +6211 4492264.96 +836 4490945.10 +895 4489141.50 +19542 4488393.75 +4322 4487884.23 +2116 4486944.65 +553 4486075.48 +2515 4485188.26 +16286 4481470.47 +12271 4478224.95 +16570 4465818.00 +7995 4457574.66 +18396 4457229.60 +16331 4455735.48 +18157 4452196.63 +5271 4452040.01 +11622 4451244.84 +4052 4446397.34 +2864 4446008.38 +490 4442892.30 +19837 4434172.39 +4114 4433657.85 +11436 4433070.15 +6085 4431306.57 +9735 4430445.60 +17834 4416286.33 +8157 4416116.65 +18840 4414925.32 +13553 4412261.70 +12562 4411183.04 +14025 4403442.16 +17964 4400360.09 +636 4399863.84 +8390 4389024.33 +231 4387397.30 +9699 4385891.02 +10622 4384005.32 +14364 4383236.90 +10580 4381533.23 +10124 4369800.96 +10451 4368867.50 +4673 4367113.44 +11351 4362616.50 +4770 4362397.32 +12932 4362042.60 +10603 4357216.50 +19733 4348931.75 +4222 4348871.91 +17319 4347687.69 +3375 4346529.48 +14995 4338295.65 +7675 4337499.60 +15043 4333921.20 +4835 4332648.00 +4408 4332588.90 +5559 4330577.09 +7376 4328936.54 +18061 4328793.98 +2749 4328671.53 +6628 4328501.88 +5888 4323049.72 +18872 4322595.62 +5476 4319642.58 +1755 4318935.63 +10623 4315822.56 +18775 4314677.64 +3570 4312697.87 +11147 4310740.57 +6071 4307612.40 +10807 4306006.00 +9550 4299478.56 +657 4296794.19 +19669 4294640.90 +8532 4290651.60 +13469 4281715.62 +8809 4280778.80 +11301 4276847.95 +6147 4266879.92 +2612 4265962.35 +15699 4256118.72 +12300 4254409.11 +3494 4250810.60 +11040 4250030.20 +6190 4244046.80 +17616 4239937.50 +7271 4234407.00 +14048 4226977.44 +4456 4224684.98 +10012 4223841.21 +11175 4223704.14 +18675 4215406.86 +10792 4214898.57 +10806 4209678.32 +18749 4204787.00 +17410 4198025.28 +8032 4195430.00 +11094 4192304.94 +17582 4187341.44 +12246 4183230.95 +6640 4182968.80 +7346 4174707.60 +12747 4169865.81 +3869 4164957.44 +13106 4161902.08 +10547 4159541.36 +15289 4156205.76 +1679 4156156.64 +1126 4155593.08 +19106 4147439.52 +9705 4144024.20 +15324 4142518.56 +16544 4140375.72 +8812 4139322.81 +10772 4134101.64 +2800 4127150.08 +15549 4124704.64 +3607 4118697.57 +1980 4117633.72 +214 4113117.36 +19217 4104217.60 +2460 4098577.46 +19156 4093864.46 +18359 4092727.29 +12865 4092526.84 +14616 4092434.54 +908 4088856.20 +11791 4083804.97 +4157 4078345.60 +3857 4070872.87 +15114 4056112.50 +395 4052997.76 +17456 4051457.28 +10562 4050894.19 +10884 4050330.76 +12177 4049842.68 +15595 4040577.56 +15916 4036044.50 +7084 4035102.72 +4424 4034761.56 +10874 4031015.85 +4740 4030403.76 +16585 4030010.26 +18824 4028984.10 +14875 4028452.08 +13855 4024828.34 +10932 4024002.40 +9084 4021362.45 +14352 4018089.74 +18086 4015180.68 +9514 4013666.67 +15787 4013154.56 +714 4010249.44 +8811 4009588.90 +14386 4007210.88 +616 4004057.26 +7460 4003412.48 +866 4003182.54 +782 4001299.94 +8562 3999441.62 +1366 3994060.86 +2879 3993056.55 +16679 3992434.99 +17306 3990723.30 +13140 3982817.39 +17942 3980857.04 +6572 3977676.28 +3578 3977523.94 +15802 3969946.90 +336 3967938.38 +9807 3964469.60 +12104 3964273.40 +4271 3962359.28 +6702 3961657.44 +19763 3955582.75 +369 3953702.88 +4089 3953455.68 +2593 3946153.80 +590 3943841.16 +8325 3942118.75 +158 3941881.65 +12054 3938362.69 +18330 3938303.88 +5354 3936239.58 +8150 3925793.46 +8344 3921293.60 +6069 3921130.55 +4032 3920008.59 +17939 3917750.27 +7014 3914471.20 +2840 3913131.58 +1868 3912987.54 +10975 3911920.48 +5374 3910802.74 +11128 3908156.46 +18449 3907589.40 +11740 3907459.84 +2356 3907189.00 +5721 3901585.97 +4231 3900779.05 +4352 3899933.44 +432 3899836.44 +15321 3899516.58 +10296 3897015.14 +5647 3895088.16 +7386 3891916.51 +507 3891487.68 +3995 3887387.07 +4278 3882294.02 +18407 3880267.86 +6127 3879166.71 +145 3875277.24 +19269 3874685.76 +18257 3874454.89 +9068 3869767.74 +576 3860007.79 +4860 3852862.02 +18793 3849838.16 +15988 3847257.05 +6891 3846386.75 +3231 3846344.30 +15237 3845421.00 +9035 3844166.85 +7597 3838643.35 +16349 3837121.65 +2497 3827850.20 +3616 3827390.95 +11566 3826122.47 +18403 3822033.02 +2972 3821903.55 +812 3821523.72 +2043 3820561.36 +505 3818922.03 +8257 3815071.92 +6084 3814194.95 +11253 3813917.24 +366 3812257.88 +13632 3811601.32 +14298 3801412.42 +7092 3798729.48 +2058 3796109.04 +14820 3791195.86 +7157 3788690.82 +17211 3786030.17 +16644 3786019.25 +15693 3783662.19 +2627 3782394.60 +11231 3782077.60 +12696 3781761.66 +8705 3778077.00 +16052 3771577.04 +99 3760269.31 +2082 3757517.50 +872 3750005.34 +7126 3749138.92 +10302 3744475.25 +17122 3741012.98 +10080 3740107.10 +16021 3739611.20 +3074 3739224.96 +3142 3738811.02 +13213 3735116.25 +13442 3733132.14 +11542 3731000.12 +13732 3730444.90 +2608 3729372.40 +5 3725511.50 +19157 3723844.72 +18231 3721707.99 +8179 3714155.04 +12740 3708646.91 +11597 3706528.59 +13968 3702376.08 +6436 3687346.44 +9181 3687134.08 +564 3680200.80 +13464 3678406.20 +14084 3673790.38 +2755 3670593.69 +14284 3668640.80 +12178 3653392.48 +15730 3650258.30 +5560 3649569.59 +8594 3647140.56 +7032 3646439.54 +16846 3644843.10 +1530 3642838.08 +3978 3639712.05 +2897 3639442.32 +16625 3636527.54 +12029 3636339.72 +16830 3633448.57 +9597 3632662.11 +5533 3630338.67 +5181 3625965.93 +8131 3625738.62 +8560 3620761.26 +11860 3618746.25 +12008 3614604.40 +10737 3611990.64 +18208 3611596.10 +5119 3611038.20 +11958 3601654.65 +15124 3598278.20 +14058 3597490.02 +12270 3593912.10 +17793 3593318.95 +9385 3587327.84 +12814 3587083.84 +5304 3586230.61 +3631 3582841.65 +610 3581917.30 +19317 3580412.43 +128 3567004.56 +11616 3566154.80 +10176 3565392.15 +7349 3564110.64 +1712 3560408.43 +18860 3559340.60 +17617 3557516.00 +6443 3556296.96 +15408 3554814.56 +16350 3554388.63 +17436 3554105.13 +5740 3551324.68 +12181 3550218.54 +16895 3550119.30 +19995 3548839.70 +4968 3548306.87 +2257 3546692.29 +1825 3543198.78 +18989 3539038.08 +18727 3536081.40 +16165 3533789.84 +3249 3533709.87 +11731 3532875.00 +13032 3532415.79 +9377 3531582.08 +5883 3531479.00 +1211 3528833.40 +12065 3526948.10 +10866 3526146.66 +2073 3520131.30 +2378 3512186.20 +16860 3509693.07 +389 3507814.64 +15604 3505653.27 +11257 3502831.80 +1327 3502022.60 +16602 3501074.88 +1493 3498808.95 +8224 3498179.52 +622 3497158.36 +3072 3495958.72 +1478 3494880.48 +3125 3494169.90 +2052 3488438.08 +8476 3487191.28 +10735 3477740.76 +14860 3476235.84 +6586 3475745.10 +5130 3472024.50 +7181 3471306.30 +618 3467906.52 +15698 3464859.47 +17585 3462450.46 +2548 3456856.96 +2632 3456230.74 +2882 3453986.86 +12216 3452907.15 +4925 3452904.63 +9012 3442581.36 +6667 3430076.40 +17958 3424962.56 +6093 3424241.92 +10648 3417414.00 +1462 3413248.61 +2569 3412388.82 +18616 3409880.91 +7368 3408036.45 +3110 3407374.60 +10824 3406819.29 +11510 3404701.96 +4840 3397236.40 +4449 3396993.60 +1358 3396616.32 +3885 3395817.60 +13381 3391953.52 +1655 3383051.51 +282 3381785.42 +4928 3374270.48 +3199 3372488.80 +16086 3370710.65 +8612 3362922.50 +19597 3360764.00 +8867 3354400.11 +4098 3353574.28 +12617 3351499.05 +14365 3347296.00 +10443 3345493.10 +76 3342081.82 +11585 3341941.22 +4383 3338960.27 +13910 3335964.16 +8076 3332449.89 +16005 3332190.40 +2622 3329364.45 +12822 3321183.52 +17076 3320398.06 +5392 3320357.15 +18628 3319615.84 +13695 3318525.99 +10326 3318274.16 +9109 3317833.90 +1489 3317620.80 +3378 3315948.00 +7738 3312979.20 +1844 3312277.36 +19963 3307500.00 +2436 3306419.05 +886 3302180.70 +15475 3301693.50 +6327 3300680.78 +6050 3299460.20 +9876 3298410.05 +19586 3291131.25 +14349 3289862.52 +10993 3287980.57 +18784 3286752.12 +1800 3285466.24 +990 3284595.50 +3823 3281992.94 +15737 3279305.96 +19518 3276759.63 +9032 3272440.32 +7786 3271217.28 +8648 3271162.44 +5532 3270187.97 +15914 3268520.98 +16065 3265068.84 +11212 3264657.03 +13229 3262022.28 +15827 3260862.72 +1582 3260340.00 +3827 3260093.76 +3546 3259244.07 +15849 3258918.00 +14856 3258379.40 +2028 3255013.96 +6618 3254581.95 +17461 3252926.88 +13551 3241602.20 +19561 3239795.32 +2276 3236172.30 +14203 3234649.39 +7757 3231351.84 +122 3226213.88 +12954 3225943.00 +647 3224783.76 +12383 3223989.44 +3831 3223126.60 +16836 3222260.73 +4565 3221597.44 +19426 3218106.54 +17855 3217813.02 +5624 3207777.36 +8368 3203376.45 +9480 3200904.00 +11181 3199500.53 +8981 3197864.00 +16426 3195995.97 +1648 3195558.90 +14404 3192729.60 +17867 3188571.00 +18117 3183229.04 +14289 3182261.60 +53 3182256.00 +15546 3180180.04 +16245 3178277.46 +1597 3176247.48 +1653 3173456.64 +2845 3171619.61 +15906 3171187.54 +18304 3168571.50 +14068 3167367.60 +6837 3165012.48 +9446 3164446.52 +18889 3156140.96 +16587 3154210.20 +7705 3152977.38 +1120 3151591.17 +17665 3148848.00 +5311 3146721.86 +14157 3144707.32 +7996 3131351.04 +8663 3130526.32 +18271 3127800.96 +6446 3125685.96 +6972 3125007.06 +2572 3123186.83 +13536 3122527.54 +6196 3122172.48 +9338 3121262.40 +11992 3118647.55 +2580 3118284.37 +9098 3117494.10 +5118 3112661.96 +10184 3109293.40 +9932 3105818.24 +18545 3102273.32 +10963 3099314.50 +8405 3097121.12 +9037 3095195.00 +179 3091107.28 +1930 3090915.80 +17723 3090624.66 +4308 3089472.75 +8702 3080129.92 +18621 3079984.80 +4501 3079781.10 +3590 3079049.42 +18264 3078858.44 +15648 3078564.06 +5998 3073264.00 +16904 3072610.80 +3794 3071333.09 +3147 3068485.32 +17221 3068337.22 +4709 3067523.31 +18017 3066743.41 +15613 3063987.86 +16271 3057051.34 +13621 3054774.59 +12919 3054518.50 +12493 3050836.30 +15838 3050645.95 +3273 3048955.15 +8324 3046011.25 +13628 3045324.50 +5522 3044408.50 +2202 3043132.05 +19052 3042566.55 +5767 3041871.70 +17895 3036452.22 +12586 3036386.30 +12425 3035041.52 +13517 3034351.47 +2363 3033336.60 +15060 3032598.51 +6764 3032591.10 +340 3030522.00 +4723 3028910.25 +3566 3027858.61 +17796 3026838.96 +15384 3023792.64 +16336 3010813.56 +679 3010713.30 +7554 3010667.80 +14553 3009756.96 +8379 3009745.17 +15436 3007499.77 +12471 3003991.86 +18059 3003037.53 +8536 3000746.00 +19033 2999373.28 +18179 2996151.20 +10711 2996143.17 +17271 2994264.79 +13932 2989023.58 +3101 2987788.16 +14550 2977853.65 +3080 2977232.58 +14533 2976490.49 +14439 2975313.24 +9237 2973124.78 +1205 2971470.28 +12361 2963419.47 +429 2962631.88 +3970 2960418.45 +8403 2957698.45 +1098 2957514.00 +7932 2955046.14 +16266 2952298.38 +19386 2948854.48 +13147 2947037.91 +2720 2947011.08 +3840 2944219.35 +13482 2942474.88 +9436 2940396.21 +19779 2937105.96 +18032 2933224.38 +7743 2932733.77 +14620 2930766.89 +4606 2927832.59 +18076 2924134.83 +19276 2918176.20 +7483 2915918.95 +8575 2915132.64 +11052 2913140.88 +17251 2908345.80 +8788 2907935.93 +10960 2906511.14 +18782 2903643.78 +19988 2897461.53 +726 2896009.27 +19942 2894251.36 +10864 2892252.48 +17840 2891563.22 +18717 2888939.96 +12391 2886051.30 +18219 2885921.06 +15100 2883342.33 +2491 2880385.74 +12389 2879696.96 +3880 2877770.24 +18579 2874542.48 +13647 2873838.34 +15758 2873804.92 +12917 2873659.60 +18866 2873616.26 +13894 2872986.12 +15200 2872571.93 +9628 2872404.56 +8568 2871598.08 +8389 2870237.88 +5788 2867210.18 +19450 2863310.66 +9440 2863162.92 +16795 2860135.41 +19643 2858987.80 +1974 2856825.84 +14622 2852089.12 +6885 2851437.62 +12532 2848992.64 +1087 2847858.80 +5777 2846407.41 +5629 2846076.12 +6316 2840544.65 +12767 2840514.12 +12134 2840036.91 +14476 2839853.01 +803 2838388.16 +18964 2836942.44 +6020 2833459.20 +10401 2832688.74 +1323 2829964.50 +1151 2829662.44 +1458 2824034.43 +2271 2820756.53 +18740 2814140.80 +7348 2811730.95 +4281 2807190.52 +8043 2804706.24 +3843 2804217.96 +7813 2802350.88 +347 2802245.52 +745 2801725.10 +10388 2799170.58 +18100 2793358.50 +19043 2789013.80 +10644 2787797.01 +16170 2787402.80 +398 2782729.05 +9370 2780078.13 +14504 2780036.04 +1210 2778485.76 +13385 2777445.62 +3799 2775223.60 +11325 2769766.02 +3489 2769554.52 +17181 2769028.50 +6964 2766653.78 +7381 2764898.80 +6253 2764394.64 +5975 2760819.72 +11996 2760687.86 +7570 2758977.12 +4387 2757672.00 +9014 2755367.42 +9403 2748021.66 +11653 2739731.07 +17697 2739312.29 +958 2738032.00 +18816 2737140.00 +14104 2735008.64 +15966 2732250.20 +17912 2724160.95 +7089 2720170.04 +16032 2718976.18 +16891 2717293.32 +19579 2716909.86 +17470 2715048.84 +12408 2712556.52 +4763 2711800.90 +1138 2709709.81 +7363 2708414.40 +7877 2705439.45 +17532 2703698.68 +10512 2701235.92 +11957 2700133.22 +2455 2699593.88 +15119 2696860.80 +9868 2696801.52 +14172 2695307.48 +16120 2689337.82 +13958 2679025.28 +15169 2676686.04 +2648 2672232.00 +6164 2671317.32 +12701 2669216.40 +16382 2669034.54 +15588 2667212.10 +14830 2666758.15 +9119 2665812.24 +1622 2665206.50 +878 2664045.79 +13269 2662784.12 +619 2655417.63 +18386 2653795.02 +2501 2652260.40 +2310 2651631.09 +19420 2649395.61 +4895 2645152.27 +7553 2643682.07 +17814 2642781.44 +16097 2642500.00 +10995 2640811.16 +14895 2637733.72 +18546 2637026.71 +9875 2631358.80 +9591 2626899.54 +6001 2625893.76 +7739 2624573.28 +10431 2624379.54 +4544 2615313.75 +16387 2603195.76 +18375 2601407.83 +8395 2598728.44 +18853 2593356.36 +4900 2592813.15 +1302 2592197.76 +17032 2589806.40 +14292 2589749.56 +43 2587359.58 +5221 2587024.04 +397 2579751.46 +17890 2579674.24 +12157 2575510.48 +7340 2574645.83 +19368 2572618.95 +8848 2570819.15 +13789 2570243.26 +14596 2568234.24 +8408 2567434.41 +19726 2565750.42 +13964 2565579.12 +7740 2563027.50 +14768 2560392.60 +11734 2559062.22 +10294 2558257.97 +15032 2557926.22 +9127 2556379.80 +2181 2553175.00 +16653 2552229.68 +3866 2549994.79 +16814 2548710.76 +1866 2545838.40 +3512 2532626.80 +4145 2529786.15 +12120 2528298.72 +644 2528123.05 +15379 2525181.01 +6392 2524063.08 +2652 2521456.80 +3363 2519202.23 +19167 2517993.18 +16042 2516599.92 +2892 2511854.40 +5711 2509401.72 +14591 2506344.69 +6564 2506277.34 +1231 2505421.24 +5049 2502603.00 +14576 2501606.69 +10211 2500852.20 +293 2493168.48 +7371 2491134.65 +18154 2491047.20 +9494 2489825.52 +14836 2480432.40 +19471 2480403.75 +802 2478998.33 +12541 2477242.60 +15065 2473563.94 +15995 2472803.20 +9408 2471953.56 +9776 2470447.90 +17325 2468989.05 +3391 2468317.72 +16123 2467022.22 +18758 2463798.06 +407 2460304.47 +6840 2456170.78 +9995 2455155.36 +3877 2453696.65 +5817 2452493.13 +14122 2452226.22 +16699 2450273.98 +8921 2450116.48 +15103 2449861.20 +7637 2449628.72 +3076 2443927.38 +6648 2443248.95 +17116 2442263.72 +1645 2440838.40 +3181 2440017.60 +5966 2431558.08 +15882 2428947.30 +7529 2428381.28 +12836 2427897.33 +18052 2427637.76 +13616 2426638.50 +16615 2424775.08 +18147 2424412.68 +4586 2424123.90 +14403 2423141.96 +11606 2422794.31 +13526 2422212.80 +3677 2421404.46 +5553 2418506.21 +12109 2416514.17 +13118 2415931.80 +1563 2408855.40 +16591 2408045.39 +6411 2404918.53 +10272 2402834.48 +10597 2400247.68 +13700 2398035.86 +9548 2397147.90 +14963 2395781.09 +13325 2390637.58 +13864 2388067.88 +7450 2383447.71 +9275 2382868.40 +5829 2378037.92 +13437 2377806.54 +13594 2375046.30 +11442 2374591.08 +15619 2374052.38 +9063 2374035.84 +5990 2368686.50 +7811 2363829.26 +9525 2362974.53 +5597 2361031.84 +8963 2360774.00 +1709 2359839.29 +15814 2358656.64 +17613 2357519.04 +5022 2354550.45 +17740 2354242.83 +3388 2351042.26 +13773 2348739.12 +14467 2348665.04 +11544 2345324.45 +349 2344664.13 +10356 2340862.72 +18272 2338754.60 +4627 2337430.84 +327 2335298.46 +19846 2332224.73 +10814 2330319.60 +13102 2326122.75 +18867 2323972.00 +2824 2323315.08 +19117 2319911.10 +1906 2319757.60 +245 2319450.90 +17318 2317860.39 +3862 2316453.72 +8100 2313874.12 +2958 2312239.47 +10263 2308514.06 +13814 2304940.40 +9394 2303161.74 +18080 2299416.78 +1271 2289526.98 +3327 2278474.48 +8740 2278405.92 +8119 2276428.17 +3368 2274373.62 +7963 2272300.80 +2151 2270932.72 +16995 2270264.68 +9918 2269733.07 +503 2268535.25 +16692 2256484.50 +793 2254198.72 +16455 2252361.86 +6644 2249521.82 +17280 2249437.50 +6813 2248982.00 +4674 2246915.32 +16325 2244369.80 +182 2243290.00 +4626 2242474.35 +10860 2241291.60 +14034 2241220.80 +2476 2240855.20 +4253 2239985.64 +3211 2239871.02 +1290 2233313.00 +8479 2232189.04 +11895 2231607.00 +3487 2230171.62 +14870 2229915.37 +16328 2229483.96 +18585 2228215.50 +7638 2228208.08 +5436 2225672.28 +14594 2223005.07 +4532 2215711.02 +7586 2210562.51 +11870 2205182.82 +18487 2203653.60 +9179 2202720.52 +16500 2201185.31 +3679 2200592.70 +12803 2198295.00 +18056 2196741.90 +11396 2195645.64 +5087 2194120.72 +8067 2192048.64 +15357 2191646.58 +4491 2189713.50 +208 2189046.80 +10958 2188766.82 +9126 2188410.50 +15084 2184327.02 +18850 2183309.52 +3398 2180250.00 +16137 2177318.76 +211 2174808.96 +18422 2174381.00 +15840 2173510.40 +19553 2173079.77 +8221 2169992.16 +17000 2169611.16 +6755 2168505.15 +10817 2167710.68 +8327 2167650.60 +543 2167368.00 +4553 2163371.52 +15019 2162288.00 +334 2162178.48 +8516 2161479.04 +11349 2158941.88 +3902 2157027.86 +14731 2155302.24 +326 2153380.08 +11403 2151242.30 +11657 2150446.08 +9496 2149219.01 +8110 2149120.13 +5153 2148527.25 +884 2148324.98 +8637 2146185.10 +2364 2145790.72 +12386 2145001.47 +10133 2144903.96 +9895 2143324.80 +13755 2142539.40 +4327 2138501.40 +3369 2137408.76 +5815 2136985.00 +19357 2132657.28 +2675 2124158.72 +17869 2123991.72 +11702 2122132.99 +17257 2117850.64 +9952 2116686.32 +3881 2111457.15 +10951 2111185.58 +2128 2109702.30 +6699 2106578.40 +3155 2103636.64 +16649 2101956.20 +15257 2100297.75 +9978 2099566.56 +16810 2098301.44 +10653 2093388.70 +10476 2092766.48 +10883 2087495.28 +9704 2086967.61 +1119 2085182.84 +19139 2079788.34 +2144 2078391.14 +9135 2076377.80 +18548 2075584.32 +10545 2075230.35 +6220 2074341.72 +8616 2072887.65 +5230 2072161.74 +13916 2070504.72 +4299 2069922.96 +894 2069688.16 +17847 2063367.04 +18879 2061902.25 +13036 2061600.17 +10606 2060492.40 +9454 2060016.48 +118 2059808.86 +9601 2059715.76 +13769 2057668.08 +1987 2057289.27 +13863 2055368.00 +13562 2054754.24 +1840 2054183.92 +17995 2053221.90 +17389 2051128.20 +15168 2045987.49 +2139 2045365.40 +4024 2044243.10 +8964 2041648.85 +181 2040167.04 +7628 2039548.92 +3 2038846.09 +15553 2036958.91 +11355 2035405.60 +13006 2034991.20 +3091 2031393.51 +1281 2030628.48 +1408 2028621.66 +18211 2024538.67 +2287 2020754.32 +6228 2019198.82 +4362 2018495.25 +10873 2013280.32 +7383 2009581.92 +1386 2006544.26 +9820 2005815.76 +18134 2003409.73 +15727 2000654.50 +157 2000148.16 +19571 1999891.11 +17728 1997944.40 +5278 1996644.21 +17737 1994653.76 +10220 1989890.98 +1397 1984509.22 +6195 1983928.26 +4270 1983726.95 +16965 1983286.25 +1683 1980638.64 +13086 1978609.40 +7124 1974039.38 +5211 1973843.76 +6794 1973149.47 +257 1973035.44 +6995 1968281.55 +8447 1967292.70 +15873 1967257.89 +12862 1964014.13 +8295 1961467.08 +931 1958825.22 +6876 1957359.48 +1932 1954592.40 +1061 1952688.06 +18108 1951143.67 +5138 1950861.00 +12598 1950211.61 +10829 1943924.62 +11950 1941211.00 +12076 1939323.96 +2176 1938691.37 +6616 1937401.88 +5893 1934358.58 +976 1933066.80 +13173 1932557.52 +14947 1929229.98 +16857 1928814.80 +13403 1928702.88 +4819 1926969.68 +13127 1926929.83 +6871 1926787.68 +15465 1925145.09 +1131 1920005.50 +11845 1913576.40 +8364 1909122.20 +16588 1904272.37 +6759 1903906.29 +11586 1901895.65 +8145 1901787.66 +17333 1897297.20 +13290 1890633.75 +6499 1887621.00 +4881 1887535.92 +7147 1886710.20 +3883 1886567.78 +18911 1885597.12 +11336 1883573.60 +8653 1883275.76 +19476 1881492.48 +14799 1880543.40 +14491 1879219.92 +11815 1877434.34 +3173 1874302.10 +7161 1873023.45 +14631 1873015.30 +4247 1869912.96 +3568 1865824.40 +1500 1865451.03 +11833 1863665.23 +495 1860771.30 +6776 1855589.17 +11374 1855221.12 +5637 1853782.17 +3597 1852826.80 +981 1852083.60 +16076 1850349.69 +17597 1845420.95 +19609 1843185.48 +10997 1843072.02 +3403 1842975.00 +897 1842845.10 +16697 1840630.68 +17644 1840597.80 +6485 1838812.02 +5492 1836202.88 +12038 1835075.06 +9325 1832634.84 +10637 1832347.44 +11318 1830158.39 +4357 1828730.00 +18553 1826335.20 +12623 1825950.85 +961 1825869.60 +1677 1821816.90 +8211 1820432.52 +19719 1819333.55 +19663 1819074.35 +16296 1818353.77 +16527 1817834.42 +4964 1815400.02 +1769 1812929.20 +13126 1808799.96 +7854 1807608.06 +18380 1803641.22 +6584 1802346.98 +7665 1801765.35 +16553 1796146.78 +17761 1795095.72 +11179 1794890.30 +15171 1794148.72 +3018 1793183.88 +15741 1788612.00 +5331 1783901.35 +9860 1775071.26 +7984 1774302.75 +15354 1774270.77 +17884 1774212.44 +16257 1771869.71 +10696 1768645.20 +2104 1767902.64 +14465 1764946.40 +10089 1764692.32 +6719 1762699.54 +3648 1760594.42 +7241 1759913.59 +11122 1757430.04 +17019 1752560.65 +13877 1744271.10 +15325 1743826.26 +17860 1739870.44 +2236 1739795.80 +4436 1738760.32 +7701 1738670.40 +8147 1736855.16 +6676 1736341.44 +19505 1735413.43 +9885 1731366.26 +2112 1725934.08 +5330 1722196.98 +3561 1720377.96 +10104 1714419.16 +16362 1712457.38 +15573 1712365.44 +15006 1711381.35 +14629 1709942.05 +9612 1709528.38 +19910 1709211.15 +13145 1708907.46 +11494 1707973.68 +15895 1706999.45 +8239 1705479.10 +2403 1705331.10 +19436 1702706.00 +3476 1702335.80 +6828 1702292.08 +771 1701589.50 +8448 1700312.44 +3755 1699047.03 +13895 1698679.03 +9785 1698056.37 +6180 1695571.53 +532 1694356.15 +6741 1692552.42 +19964 1692367.64 +3747 1691244.60 +3253 1690719.42 +16119 1688339.25 +7113 1681911.00 +12368 1681219.80 +16378 1679705.60 +1393 1675545.35 +11119 1675453.44 +4469 1674023.49 +6955 1672618.90 +11579 1672345.32 +19898 1671781.70 +15351 1659204.30 +6133 1658215.46 +9110 1658054.68 +2979 1656016.74 +18764 1653708.48 +8995 1653627.58 +13096 1651408.67 +15062 1650548.02 +7924 1650202.40 +10076 1647970.24 +15859 1646036.28 +17932 1642640.66 +19694 1642089.50 +13827 1642001.31 +17963 1639689.00 +10698 1635848.26 +18003 1633530.78 +8416 1633366.77 +476 1631154.06 +2806 1630782.80 +12129 1628615.47 +11215 1626624.70 +14061 1624933.44 +5956 1623586.10 +9043 1622670.40 +13287 1621980.36 +11410 1621420.90 +13990 1621268.20 +12952 1619215.18 +15181 1619088.68 +9784 1618120.53 +10733 1616168.88 +16054 1614531.23 +5864 1614397.83 +1875 1611927.00 +17381 1611664.80 +14562 1607467.92 +575 1605941.73 +2005 1605591.72 +4332 1605448.83 +4653 1602596.30 +15403 1601830.44 +17430 1599681.42 +4798 1593630.50 +12991 1593321.52 +15653 1593138.66 +10066 1593049.06 +8892 1592100.90 +6708 1590159.12 +9825 1589403.92 +8271 1588475.41 +17084 1584280.88 +4003 1583631.00 +869 1582643.16 +16400 1582313.20 +19088 1581708.56 +6581 1581346.80 +9481 1581048.60 +6092 1580846.49 +3624 1578777.30 +6503 1578507.78 +14557 1578280.96 +2428 1577543.92 +15513 1573560.21 +4641 1573363.54 +10152 1570213.60 +5932 1566902.52 +7482 1561323.50 +13745 1558358.34 +2251 1558274.70 +9845 1558068.12 +7603 1557388.20 +1809 1553837.20 +18128 1547643.36 +8086 1543199.04 +14948 1541721.57 +16725 1540948.50 +2999 1540317.66 +8861 1540008.47 +1964 1538815.25 +19374 1537884.78 +15428 1535994.36 +7449 1534782.48 +16884 1534509.16 +10271 1534397.34 +11782 1529963.22 +8184 1529750.70 +4560 1527433.24 +4616 1525374.46 +3814 1524077.04 +17265 1523932.08 +16520 1522906.28 +10475 1518705.06 +5094 1517317.83 +8626 1515142.07 +19895 1512286.68 +19933 1506235.36 +6854 1505626.00 +13995 1505562.18 +7102 1504945.67 +9079 1501237.20 +18329 1500146.90 +3742 1496990.77 +12395 1496904.43 +12214 1496489.40 +12298 1495554.30 +4978 1495389.50 +2927 1494280.10 +2119 1494151.14 +15143 1492039.75 +14548 1487406.60 +840 1486128.98 +5902 1486097.28 +10614 1482144.72 +5895 1481356.80 +15958 1480951.60 +11408 1479948.96 +8407 1474236.00 +6243 1471007.85 +10389 1469004.46 +13871 1468938.64 +19811 1464597.09 +10495 1464290.49 +4389 1463010.83 +1311 1461703.36 +17874 1459408.88 +6597 1458761.87 +19211 1456741.63 +12879 1456178.24 +8840 1455731.46 +14755 1454890.60 +16957 1454465.96 +9257 1454388.76 +5193 1454011.32 +6884 1452474.60 +19948 1452024.00 +15076 1448395.00 +16016 1447557.45 +11693 1445839.68 +6975 1440516.96 +4290 1439768.46 +18900 1438722.10 +14383 1438477.92 +15098 1435941.78 +9322 1435282.80 +458 1433040.45 +10042 1432906.35 +5052 1431900.90 +6600 1431116.55 +3630 1428665.04 +9636 1428193.84 +16511 1427308.74 +4045 1427248.35 +19562 1426348.82 +8814 1425690.09 +2616 1425178.04 +4587 1425109.40 +148 1424237.37 +2712 1423780.26 +10863 1423386.16 +16096 1421942.09 +18936 1421938.65 +18327 1419872.92 +11620 1419050.10 +3740 1418609.85 +3457 1418603.50 +1185 1417637.47 +8178 1417357.26 +17791 1413293.13 +13608 1411323.12 +17849 1409613.50 +6814 1406228.40 +14022 1406138.04 +14231 1403771.52 +19546 1402854.60 +19619 1402389.16 +5609 1402302.54 +5342 1401567.59 +3084 1401096.10 +5708 1400334.90 +17998 1399862.45 +19850 1397630.33 +14004 1395443.10 +13071 1394653.24 +2797 1393747.58 +2866 1392947.25 +19809 1389067.68 +13600 1380865.80 +13614 1380654.36 +5884 1380319.74 +9404 1378623.66 +10656 1376954.32 +12324 1376502.40 +7325 1375030.43 +13295 1373987.34 +11864 1373555.68 +6987 1373481.51 +8386 1371854.41 +10916 1370374.32 +12867 1369058.11 +14668 1369040.34 +13383 1367342.30 +18572 1366953.96 +1152 1366861.38 +6015 1366452.18 +3344 1366185.15 +7889 1365521.92 +13345 1364088.88 +6276 1363421.62 +8069 1361824.20 +17509 1360892.49 +15137 1358678.07 +17163 1357391.52 +4704 1356692.40 +8609 1356578.19 +12644 1356088.14 +17141 1356022.38 +11805 1354826.78 +6386 1354187.22 +3004 1352173.44 +8634 1350211.80 +4399 1349881.20 +10362 1349411.34 +1572 1348835.20 +7359 1348224.10 +11884 1346696.82 +11671 1346424.15 +5350 1346359.28 +3119 1345996.48 +5307 1345356.00 +16117 1345045.12 +8715 1342665.72 +5398 1341179.28 +7627 1338820.56 +8457 1337714.68 +4958 1334732.71 +84 1334146.71 +6932 1333235.36 +757 1332921.07 +4076 1332441.00 +1751 1329112.32 +15701 1327052.16 +4119 1326549.90 +1562 1325604.28 +8741 1325517.60 +1135 1325422.71 +1002 1323418.65 +5832 1323085.71 +5368 1322793.96 +5382 1322628.84 +5616 1319082.26 +2832 1318691.95 +3895 1317858.44 +8629 1317756.51 +5709 1317058.68 +18383 1316451.05 +15797 1314806.64 +1900 1313660.40 +13882 1310455.86 +6785 1309877.80 +14855 1309280.76 +7761 1308602.24 +14268 1306810.40 +6257 1306056.96 +19002 1305509.52 +5095 1303729.02 +10320 1301657.58 +7826 1299561.68 +13359 1298717.14 +7436 1298127.36 +5644 1295055.77 +11327 1290526.41 +5277 1289329.65 +15932 1286235.84 +14322 1284809.36 +144 1284270.12 +3043 1281162.79 +16788 1280955.34 +17136 1280443.12 +12560 1279117.95 +13833 1278834.75 +5414 1277893.26 +12582 1277592.32 +4644 1277535.00 +14032 1277077.88 +18325 1271719.68 +7072 1271228.48 +16868 1267469.42 +8137 1267425.81 +5976 1266206.85 +14125 1265569.05 +13299 1265287.55 +18376 1264249.30 +6157 1261759.92 +5002 1261669.64 +13368 1260918.60 +15589 1260059.76 +2149 1258981.44 +9639 1256283.38 +11689 1256027.92 +9083 1245924.24 +16231 1242625.65 +5084 1242385.28 +11634 1240760.18 +15617 1239731.25 +9865 1237181.62 +14212 1236365.52 +10325 1235223.36 +19582 1235105.76 +740 1234746.81 +19231 1233623.10 +16840 1233063.85 +5703 1231744.33 +5761 1229435.20 +15630 1226611.62 +10408 1224698.40 +9177 1221942.51 +13389 1221666.75 +6104 1221577.92 +9673 1218826.64 +2707 1217124.48 +18672 1214208.80 +5112 1209590.20 +6264 1208318.50 +18496 1207881.75 +10971 1207183.52 +19059 1206729.90 +431 1205938.44 +3821 1201192.75 +826 1200454.62 +3317 1200440.90 +19689 1198899.52 +19641 1198797.99 +6379 1197195.50 +814 1194417.40 +18643 1194000.78 +11865 1193965.76 +12393 1193896.80 +9218 1193660.58 +8674 1191881.32 +8582 1191804.02 +13084 1191508.00 +18844 1190239.96 +16061 1189935.00 +6134 1185550.80 +8628 1183245.60 +8884 1181547.48 +7697 1181032.50 +9044 1180922.60 +13257 1180158.57 +8066 1178808.12 +5876 1177376.80 +14694 1177059.31 +16062 1175391.00 +9104 1175178.90 +11600 1175091.06 +10337 1172684.92 +19188 1172349.78 +8833 1171372.93 +6895 1170602.07 +14100 1168878.40 +13538 1168554.28 +3408 1166645.16 +1860 1165673.68 +13436 1164278.70 +19325 1162733.70 +7403 1161982.00 +4882 1161404.81 +13105 1161320.58 +17880 1161256.02 +19284 1160927.60 +13476 1159035.15 +18913 1158208.30 +18523 1158135.00 +12508 1157538.45 +9090 1156362.64 +17653 1154338.08 +3926 1152652.52 +10183 1148324.57 +7556 1146268.14 +16436 1142656.47 +4741 1141614.00 +15651 1141497.93 +3183 1140081.36 +9532 1139902.50 +16403 1139306.37 +2368 1137421.16 +3889 1136395.50 +2885 1135838.14 +7851 1135110.76 +16234 1135017.24 +12746 1134531.04 +2647 1132941.12 +5373 1132158.01 +10340 1132004.24 +8873 1131949.28 +1132 1131338.88 +15594 1131328.62 +4376 1130282.20 +240 1126682.48 +2231 1124447.15 +929 1121383.92 +11599 1119307.27 +3765 1119093.50 +17635 1118420.16 +7119 1118285.08 +15121 1117715.34 +11858 1116963.54 +16963 1116929.45 +16356 1113648.98 +6924 1112198.40 +16223 1111257.00 +18091 1110043.02 +12628 1108954.80 +16043 1108831.05 +9402 1108290.48 +708 1107084.00 +4078 1105993.96 +17593 1104713.40 +12776 1104362.59 +7583 1102813.53 +14619 1102675.80 +8842 1100110.26 +4196 1099726.55 +2019 1098178.64 +6863 1097246.36 +6489 1096503.07 +2459 1094813.04 +11964 1094485.02 +3236 1093969.80 +17647 1093809.15 +17648 1093114.62 +119 1092687.48 +9626 1092080.00 +9124 1091569.68 +13175 1089851.76 +2532 1088706.35 +16083 1088295.39 +8874 1086011.34 +12872 1082970.30 +19821 1082520.84 +4800 1080389.70 +18696 1079685.36 +19545 1079184.33 +13120 1077742.28 +10588 1076203.83 +17696 1075092.72 +14651 1073222.23 +903 1071146.76 +5858 1070259.48 +8302 1069504.80 +18728 1069225.51 +18026 1068569.00 +19383 1066907.58 +18690 1065930.90 +5924 1065143.12 +4880 1065011.75 +12439 1064381.19 +16529 1062371.70 +19653 1057683.56 +3136 1056810.44 +18932 1056193.65 +2124 1054160.52 +16851 1052646.84 +10123 1051624.00 +5618 1048447.93 +19851 1045187.85 +16278 1044808.38 +11479 1044276.22 +13263 1042046.20 +6041 1041123.38 +7193 1040455.32 +19408 1039430.01 +11260 1036828.52 +5179 1035633.44 +1331 1034398.00 +7706 1034249.40 +8436 1033549.35 +1801 1031886.00 +4170 1031642.90 +11827 1031139.39 +17114 1027985.88 +18278 1026583.11 +1995 1025165.68 +7667 1022980.15 +6559 1021635.45 +17488 1021612.13 +16059 1019781.19 +7633 1018782.57 +10032 1016809.50 +2899 1016438.76 +14628 1016033.20 +10126 1015846.78 +3884 1014413.50 +16913 1013604.40 +18644 1010288.10 +19870 1007919.36 +18564 1007416.20 +10179 1004920.00 +883 1004650.68 +3627 1004461.04 -- !select -- -12098 16227681.210000000 -5134 15709338.520000000 -13334 15023662.410000000 -17052 14351644.200000000 -3452 14070870.140000000 -12552 13332469.180000000 -1084 13170428.290000000 -5797 13038622.720000000 -12633 12892561.610000000 -403 12856217.340000000 -1833 12024581.720000000 -2084 11502875.360000000 -17349 11354213.050000000 -18427 11282385.240000000 -2860 11262529.950000000 -17852 10934711.930000000 -9871 10889253.680000000 -12231 10841131.390000000 -6366 10759786.810000000 -12146 10257362.660000000 -5043 10226395.880000000 -12969 10125777.930000000 -1504 10004397.080000000 -14327 9981697.080000000 -134 9965150.660000000 -6860 9805871.260000000 -10624 9776138.400000000 -15819 9775705.310000000 -3293 9674928.120000000 -19865 9653766.830000000 -8870 9648981.870000000 -15778 9636332.820000000 -12360 9635023.920000000 -14389 9475588.340000000 -3257 9451029.240000000 -9476 9435207.280000000 -19629 9391236.400000000 -7179 9386222.250000000 -15723 9383900.800000000 -4054 9313810.020000000 -2380 9307751.220000000 -19084 9302916.800000000 -4703 9280804.800000000 -18791 9267017.970000000 -19994 9235972.920000000 -9149 9121803.900000000 -15118 9120819.500000000 -6116 9079369.200000000 -7052 9077468.920000000 -14147 9069193.780000000 -7305 9035228.530000000 -9130 9024379.250000000 -16698 8991337.950000000 -1553 8977226.100000000 -16777 8961355.620000000 -1402 8953779.120000000 -18963 8934063.400000000 -8358 8930611.480000000 -17547 8860117.000000000 -5128 8844222.750000000 -17063 8840649.600000000 -15490 8833581.400000000 -14761 8817240.560000000 -19601 8791341.020000000 -16160 8740262.760000000 -13597 8702669.820000000 -13653 8693170.160000000 -16383 8691505.920000000 -325 8667741.280000000 -8879 8667584.380000000 -10564 8667098.220000000 -17429 8661827.900000000 -17403 8643350.300000000 -18294 8616583.430000000 -4181 8592684.660000000 -13008 8567480.640000000 -13211 8537000.010000000 -1884 8532644.340000000 -11101 8530945.320000000 -11562 8528028.570000000 -15878 8523591.840000000 -834 8522135.270000000 -2423 8517902.850000000 -15383 8513433.110000000 -18119 8507611.800000000 -7389 8506099.200000000 -5016 8489784.150000000 -17473 8444766.240000000 -6669 8428618.460000000 -384 8418472.270000000 -12052 8411519.280000000 -17562 8409022.830000000 -8128 8379149.470000000 -13813 8374830.840000000 -12800 8318626.780000000 -10887 8315019.360000000 -1644 8285453.080000000 -16638 8274568.000000000 -1394 8255140.600000000 -7219 8254985.300000000 -13358 8253829.800000000 -5562 8252365.160000000 -14861 8242296.150000000 -15416 8196621.530000000 -1963 8192206.610000000 -2841 8148678.450000000 -6635 8122918.280000000 -3609 8099812.050000000 -6372 8093695.060000000 -5231 8091704.180000000 -8755 8085017.030000000 -4071 8083755.180000000 -4977 8058501.720000000 -11873 8057540.300000000 -12127 8051142.940000000 -2401 8049525.940000000 -15964 8037547.750000000 -10129 8030855.040000000 -7605 8028622.420000000 -9327 8022291.300000000 -11814 7983589.590000000 -4646 7981660.600000000 -6032 7981511.280000000 -1076 7977074.750000000 -4043 7971929.900000000 -8932 7967222.190000000 -13705 7953344.520000000 -16855 7923819.000000000 -3234 7920022.000000000 -17752 7901362.770000000 -2097 7892993.170000000 -18599 7890774.400000000 -19596 7874564.590000000 -11392 7861172.480000000 -18121 7857581.600000000 -17452 7838465.360000000 -6076 7821316.800000000 -15134 7804760.250000000 -8641 7802917.150000000 -2134 7800185.430000000 -16352 7797112.000000000 -19312 7775952.240000000 -2895 7759327.230000000 -12860 7758135.210000000 -153 7755681.280000000 -15089 7735438.260000000 -14797 7725353.160000000 -15946 7722773.880000000 -10919 7722425.360000000 -9867 7721597.780000000 -11881 7713136.420000000 -16552 7708518.150000000 -6925 7703999.680000000 -12147 7703826.980000000 -8923 7702690.280000000 -16116 7697970.840000000 -10661 7673830.200000000 -17094 7665368.160000000 -18648 7650862.020000000 -12172 7641326.400000000 -15123 7633032.500000000 -4993 7600570.800000000 -17162 7592062.560000000 -13506 7580809.830000000 -3436 7575616.330000000 -6271 7559793.930000000 -13314 7555156.630000000 -17242 7550949.500000000 -2753 7549574.060000000 -7391 7543159.680000000 -7418 7541449.650000000 -116 7520874.240000000 -12436 7520234.540000000 -1181 7494798.280000000 -12963 7491248.100000000 -213 7479470.280000000 -7114 7477681.200000000 -18521 7476478.300000000 -8973 7458603.670000000 -4202 7454095.740000000 -12009 7442105.400000000 -10609 7429346.400000000 -5622 7424142.660000000 -5143 7422760.280000000 -898 7414133.800000000 -12257 7408190.630000000 -6740 7400350.350000000 -1146 7394394.480000000 -5485 7378181.940000000 -8437 7376353.590000000 -6531 7362366.780000000 -16463 7362106.800000000 -10412 7359552.420000000 -12857 7340801.580000000 -12354 7332343.200000000 -7616 7320032.370000000 -3426 7312340.410000000 -8622 7307266.330000000 -6818 7304782.100000000 -3705 7299380.100000000 -12733 7298398.170000000 -1574 7293446.040000000 -10554 7289933.720000000 -9363 7284647.700000000 -4611 7282115.000000000 -7857 7266948.920000000 -9616 7265005.300000000 -15860 7254466.660000000 -15554 7247592.510000000 -3422 7247586.450000000 -9134 7236404.670000000 -17408 7220083.600000000 -15853 7219984.900000000 -9266 7218517.110000000 -1881 7208346.450000000 -10148 7205335.830000000 -8860 7202401.440000000 -8373 7189039.500000000 -10859 7188990.690000000 -12670 7188177.220000000 -2553 7180006.020000000 -19346 7176797.250000000 -1681 7160169.840000000 -15225 7158861.320000000 -1052 7158586.000000000 -77 7155531.100000000 -7231 7155250.380000000 -12622 7137408.420000000 -9814 7105363.140000000 -8695 7103187.000000000 -13174 7099182.530000000 -6179 7095134.050000000 -6451 7082495.360000000 -19860 7073206.830000000 -9307 7059973.680000000 -7819 7055963.040000000 -10556 7053491.070000000 -9366 7048690.740000000 -12124 7040021.310000000 -11476 7037906.760000000 -19245 7034045.240000000 -7562 7030275.790000000 -12290 7020372.060000000 -18118 7003396.800000000 -1253 7001569.620000000 -8662 6999834.270000000 -1779 6997385.730000000 -15386 6996871.790000000 -147 6989079.880000000 -9562 6983076.900000000 -3958 6969833.220000000 -7211 6966606.670000000 -12842 6923277.660000000 -4368 6918783.950000000 -11131 6918656.780000000 -4628 6894893.820000000 -5879 6881367.140000000 -16586 6865931.800000000 -32 6852925.590000000 -12119 6842773.700000000 -1371 6831137.520000000 -6136 6827917.010000000 -13857 6824240.600000000 -1074 6821747.880000000 -1863 6821522.190000000 -14597 6817385.660000000 -9271 6783068.880000000 -6389 6781075.680000000 -11703 6776538.360000000 -13701 6768880.560000000 -14880 6763788.240000000 -18428 6763670.540000000 -1006 6762065.940000000 -4927 6756765.210000000 -11659 6755246.680000000 -6815 6738928.350000000 -14367 6733857.200000000 -10703 6730936.460000000 -3150 6727920.400000000 -9963 6725919.350000000 -10438 6710153.620000000 -4745 6682153.670000000 -15297 6681711.280000000 -10848 6678666.250000000 -11749 6677895.730000000 -16739 6675549.120000000 -14915 6672248.660000000 -19841 6669191.200000000 -628 6666892.900000000 -1650 6657178.320000000 -7069 6648672.240000000 -7108 6646445.960000000 -8887 6641655.900000000 -18735 6636457.560000000 -3711 6632665.380000000 -2325 6630475.920000000 -6484 6622965.200000000 -2643 6617304.980000000 -7895 6615061.260000000 -12840 6604848.540000000 -4314 6600905.710000000 -19077 6591091.970000000 -17476 6576029.000000000 -7039 6559838.900000000 -8106 6558223.020000000 -2391 6557120.080000000 -7717 6547706.880000000 -12476 6546988.150000000 -9155 6540793.600000000 -360 6530297.410000000 -6383 6529336.020000000 -4830 6518998.920000000 -12600 6511549.460000000 -13740 6508057.920000000 -4678 6507847.980000000 -17815 6502284.760000000 -7329 6490811.950000000 -13884 6490063.100000000 -5147 6487069.000000000 -16548 6482024.500000000 -14144 6476413.400000000 -10181 6474984.880000000 -11031 6463308.020000000 -19958 6461506.380000000 -6043 6458177.640000000 -10060 6455476.890000000 -9144 6454042.050000000 -7043 6448019.980000000 -17346 6444307.520000000 -13963 6442014.480000000 -7111 6441947.070000000 -14140 6439955.540000000 -2327 6438977.280000000 -14812 6437152.540000000 -16755 6430895.360000000 -14840 6430549.140000000 -14134 6422079.260000000 -12655 6405496.790000000 -1518 6390148.220000000 -9888 6385033.020000000 -7387 6384005.180000000 -5393 6381083.040000000 -11057 6375974.220000000 -16818 6368828.800000000 -4576 6364925.710000000 -19644 6350000.330000000 -139 6336065.550000000 -11735 6334305.900000000 -10108 6332055.990000000 -15628 6329801.340000000 -4349 6314949.680000000 -7048 6313868.550000000 -17119 6298935.380000000 -18597 6291416.310000000 -2488 6286372.050000000 -2296 6275519.250000000 -4343 6272834.200000000 -9958 6267714.610000000 -2147 6267475.320000000 -9368 6254578.920000000 -13134 6252432.760000000 -10119 6251456.250000000 -2093 6249342.360000000 -2392 6237476.560000000 -17231 6233509.180000000 -3773 6226654.680000000 -9839 6214044.100000000 -19349 6213969.300000000 -7869 6212751.000000000 -8158 6210279.240000000 -13179 6205773.650000000 -2349 6205589.260000000 -9052 6200729.700000000 -1744 6189967.360000000 -597 6183103.470000000 -16721 6176606.600000000 -7498 6176277.250000000 -15157 6176222.500000000 -17524 6171107.360000000 -7922 6170906.070000000 -6615 6170738.420000000 -2706 6165991.650000000 -16432 6162740.680000000 -16473 6162427.960000000 -12825 6161595.600000000 -18813 6154678.550000000 -1030 6146500.280000000 -2571 6145772.430000000 -7707 6144754.710000000 -10327 6137612.000000000 -4710 6132346.560000000 -3649 6130602.630000000 -4893 6128461.240000000 -12844 6128191.240000000 -8794 6122690.280000000 -1157 6117749.220000000 -10895 6112017.680000000 -16166 6108250.980000000 -11920 6107122.560000000 -5621 6102123.560000000 -6141 6093826.560000000 -14076 6086671.080000000 -15884 6080485.590000000 -4814 6080337.960000000 -5814 6079842.960000000 -1134 6078685.200000000 -19048 6071813.280000000 -303 6070601.350000000 -15211 6065678.060000000 -1020 6054168.000000000 -11292 6052522.830000000 -7134 6049865.250000000 -14119 6049619.550000000 -2389 6042429.120000000 -5926 6034269.520000000 -8553 6030922.950000000 -18814 6023255.170000000 -12702 6023190.990000000 -2644 6020931.330000000 -19628 6010704.000000000 -18112 6008707.520000000 -13860 6008170.290000000 -1456 6005092.140000000 -1633 6002068.850000000 -2301 6000302.290000000 -10740 5999809.050000000 -2630 5997008.430000000 -8818 5992296.900000000 -10043 5990594.020000000 -653 5987942.830000000 -6829 5985990.660000000 -15179 5977727.520000000 -9663 5973523.920000000 -5863 5973328.920000000 -3628 5966340.090000000 -7618 5960155.860000000 -2588 5952648.560000000 -4865 5949383.400000000 -4233 5944699.600000000 -13390 5944104.690000000 -16321 5942714.700000000 -9653 5941308.500000000 -18884 5925548.240000000 -4394 5920927.140000000 -19774 5916723.120000000 -1257 5914052.360000000 -2963 5911917.770000000 -17157 5899573.020000000 -383 5884693.360000000 -11709 5884134.480000000 -18135 5871431.610000000 -13169 5869736.880000000 -2932 5868995.800000000 -2888 5863229.300000000 -6820 5853106.800000000 -18622 5850951.600000000 -9066 5846052.720000000 -19840 5832251.400000000 -6325 5827298.430000000 -14092 5823711.720000000 -11532 5823277.120000000 -18043 5815248.700000000 -3932 5809134.600000000 -10364 5808371.400000000 -1179 5808303.240000000 -11441 5799378.330000000 -15767 5798263.260000000 -14063 5797204.000000000 -11383 5793893.120000000 -10065 5781764.450000000 -17785 5766838.760000000 -18349 5761765.770000000 -14824 5760339.780000000 -14699 5759490.180000000 -11628 5755137.200000000 -4403 5752376.640000000 -13090 5751949.250000000 -15332 5744606.280000000 -17573 5744384.150000000 -12895 5741007.020000000 -13577 5739209.940000000 -16594 5732523.640000000 -8497 5727509.840000000 -2872 5724068.720000000 -16181 5721788.160000000 -6639 5712041.220000000 -13095 5708798.250000000 -4535 5693341.200000000 -10956 5692585.500000000 -19755 5686913.370000000 -12995 5682200.410000000 -13157 5681454.940000000 -1887 5681341.240000000 -18472 5680356.000000000 -19814 5679360.000000000 -18583 5669360.520000000 -3894 5664385.490000000 -1740 5659552.040000000 -62 5659470.160000000 -16532 5653779.460000000 -995 5648973.450000000 -7486 5646851.870000000 -19007 5642710.930000000 -13060 5642024.640000000 -12371 5635710.450000000 -2280 5634077.400000000 -3115 5631352.320000000 -11107 5631252.460000000 -5873 5629125.750000000 -14743 5628732.800000000 -2224 5624746.620000000 -2653 5623859.990000000 -17622 5623057.440000000 -14700 5615344.000000000 -14740 5613929.390000000 -6403 5611924.510000000 -6896 5609169.880000000 -10384 5607337.520000000 -16433 5605707.030000000 -5898 5604937.760000000 -4789 5600488.360000000 -8132 5593107.500000000 -3838 5592313.200000000 -13631 5586424.800000000 -11233 5585247.010000000 -849 5583516.450000000 -14653 5581550.450000000 -14788 5580433.000000000 -18181 5578562.880000000 -19815 5577102.620000000 -5584 5576692.200000000 -5385 5576420.190000000 -13780 5569028.610000000 -9342 5566783.710000000 -19056 5566524.120000000 -8189 5565694.740000000 -13808 5560721.960000000 -10635 5560058.550000000 -8304 5550784.410000000 -14257 5549164.060000000 -8999 5542100.100000000 -19134 5539312.560000000 -8360 5538031.100000000 -6397 5536651.920000000 -2597 5525317.760000000 -8631 5515909.380000000 -16729 5512663.650000000 -11861 5511785.930000000 -16853 5511689.910000000 -6341 5502790.080000000 -1312 5496649.400000000 -5566 5495885.870000000 -12519 5490649.970000000 -19032 5488105.020000000 -8231 5479312.500000000 -3026 5466732.320000000 -6388 5466168.800000000 -15349 5464571.520000000 -18985 5463897.130000000 -19848 5454266.400000000 -17378 5453284.740000000 -4000 5448690.390000000 -3710 5445822.530000000 -13181 5439774.060000000 -6420 5438325.320000000 -3644 5437772.140000000 -1117 5437024.970000000 -13027 5436968.460000000 -12884 5432632.340000000 -12781 5429161.080000000 -10084 5428231.620000000 -5640 5423318.580000000 -16208 5422901.400000000 -753 5416899.740000000 -4120 5413822.460000000 -12348 5412061.710000000 -1586 5411198.550000000 -2503 5411125.800000000 -1381 5397790.100000000 -19673 5397746.770000000 -19389 5394426.400000000 -15532 5386361.380000000 -3562 5380335.080000000 -19114 5375133.000000000 -3204 5372215.460000000 -6653 5365178.000000000 -3553 5363690.290000000 -12687 5361682.120000000 -3279 5357505.650000000 -9840 5350112.180000000 -8258 5347064.670000000 -11387 5345284.000000000 -15210 5341117.920000000 -15092 5340896.800000000 -6052 5339762.180000000 -14043 5339587.680000000 -6284 5336153.000000000 -6297 5332357.740000000 -16254 5326607.830000000 -18436 5326198.500000000 -14436 5325517.140000000 -10368 5319983.080000000 -6349 5317943.400000000 -19984 5317620.000000000 -19080 5310678.600000000 -1403 5306771.320000000 -5995 5305056.120000000 -13214 5299420.450000000 -19293 5297054.400000000 -7130 5289761.480000000 -9809 5286784.800000000 -9273 5277924.770000000 -16786 5260035.000000000 -11032 5256354.350000000 -17719 5246322.200000000 -3400 5244203.680000000 -8278 5243718.480000000 -7873 5241168.800000000 -9357 5239695.770000000 -2239 5235767.830000000 -18695 5223960.350000000 -19271 5216498.290000000 -4563 5214673.700000000 -1896 5210699.790000000 -15755 5210481.240000000 -17628 5209882.370000000 -5747 5206892.240000000 -18057 5204907.400000000 -5075 5204412.810000000 -5325 5190162.750000000 -17675 5183135.360000000 -18818 5181295.300000000 -11064 5180667.570000000 -12578 5177222.330000000 -3725 5169183.750000000 -11112 5161546.000000000 -4749 5161499.440000000 -7632 5159180.500000000 -13119 5151665.510000000 -5653 5140324.640000000 -16134 5137482.510000000 -5671 5136815.550000000 -18348 5132981.070000000 -562 5132487.250000000 -2562 5126763.830000000 -10304 5125180.000000000 -7622 5120719.980000000 -12755 5118213.920000000 -1814 5100705.580000000 -11269 5096671.330000000 -2964 5095253.720000000 -1616 5091834.000000000 -9294 5090753.530000000 -16793 5085330.540000000 -4999 5081651.750000000 -17209 5079029.280000000 -7151 5078937.600000000 -15522 5072469.600000000 -3056 5072329.550000000 -7612 5068322.870000000 -18453 5063892.920000000 -18324 5058901.220000000 -12266 5058186.750000000 -19394 5056235.730000000 -1713 5054968.050000000 -15681 5051569.630000000 -8274 5043328.000000000 -18160 5043074.830000000 -18253 5041572.000000000 -11840 5040590.040000000 -1532 5033171.000000000 -584 5031602.640000000 -12382 5028901.000000000 -14814 5022200.070000000 -19058 5019001.920000000 -4487 5016640.860000000 -8482 5015444.250000000 -18476 5011136.360000000 -12335 5003581.400000000 -4455 4997933.310000000 -14355 4992822.920000000 -15253 4992642.200000000 -14069 4983244.200000000 -17843 4977294.370000000 -9389 4975885.830000000 -14435 4971442.190000000 -13254 4959481.450000000 -9773 4955887.800000000 -7615 4952421.540000000 -6476 4947250.050000000 -9253 4945159.700000000 -14932 4934395.480000000 -13253 4932867.450000000 -19322 4931525.780000000 -16945 4931440.610000000 -731 4930191.930000000 -6540 4925114.510000000 -5148 4923048.000000000 -1934 4921196.900000000 -15402 4920840.720000000 -17914 4919607.040000000 -5416 4916041.920000000 -16734 4914205.270000000 -14967 4900262.080000000 -8706 4894595.580000000 -136 4891960.920000000 -19494 4886028.300000000 -8737 4880640.440000000 -7653 4879423.640000000 -4149 4875782.400000000 -7890 4872424.400000000 -11142 4871415.420000000 -10386 4863623.500000000 -8603 4861814.610000000 -2680 4861223.740000000 -4891 4858103.880000000 -19236 4855097.690000000 -14251 4854739.860000000 -18895 4853365.100000000 -17134 4852041.040000000 -4932 4843701.450000000 -10033 4841647.720000000 -1383 4839144.310000000 -18721 4837983.360000000 -8618 4833125.420000000 -17386 4831545.840000000 -3790 4830282.360000000 -1043 4825921.310000000 -12434 4822597.780000000 -18385 4819643.400000000 -6046 4817460.060000000 -5821 4814423.450000000 -10836 4814303.240000000 -6848 4813954.080000000 -6880 4804600.350000000 -11249 4800116.820000000 -11970 4799739.660000000 -14253 4796521.290000000 -7782 4793227.130000000 -75 4790042.880000000 -7076 4789347.340000000 -9566 4782531.800000000 -2137 4767931.740000000 -2336 4763870.790000000 -15362 4759043.380000000 -3284 4755048.760000000 -12964 4753627.480000000 -1781 4752835.200000000 -5454 4748342.980000000 -12597 4742077.840000000 -19120 4733459.960000000 -14884 4731499.440000000 -825 4730720.280000000 -14683 4730482.320000000 -5361 4726113.000000000 -12179 4725018.100000000 -1461 4710954.690000000 -9890 4709658.400000000 -13369 4705085.390000000 -11612 4701627.990000000 -3096 4699414.400000000 -10383 4697866.470000000 -11293 4697336.060000000 -3383 4695825.200000000 -6231 4694381.720000000 -7396 4691319.060000000 -17827 4688797.440000000 -15856 4683368.210000000 -8253 4678560.860000000 -12327 4677984.740000000 -4395 4676829.820000000 -4232 4676646.400000000 -14260 4670522.800000000 -15288 4669273.990000000 -17526 4668545.640000000 -9884 4662693.840000000 -2118 4660352.780000000 -4524 4653956.600000000 -19090 4650872.940000000 -3928 4649359.440000000 -14325 4647762.170000000 -15476 4643469.040000000 -4179 4639931.760000000 -14408 4639631.280000000 -19424 4634817.440000000 -3334 4633102.500000000 -9477 4628073.510000000 -11803 4625396.800000000 -14805 4618452.180000000 -463 4616307.280000000 -16628 4607490.960000000 -3116 4604463.100000000 -19962 4602949.470000000 -12859 4602870.550000000 -12063 4600708.450000000 -5648 4592273.250000000 -8556 4590726.860000000 -15281 4589425.410000000 -9414 4587426.900000000 -13951 4586281.250000000 -19328 4582624.820000000 -15963 4579705.500000000 -10773 4573276.200000000 -14179 4568816.000000000 -1895 4563988.160000000 -6408 4561496.390000000 -5958 4554000.000000000 -3653 4548134.400000000 -11218 4546237.920000000 -19327 4543987.770000000 -9572 4535941.160000000 -14556 4531464.750000000 -2475 4529761.500000000 -9631 4529261.560000000 -1901 4528592.550000000 -86 4528475.380000000 -9586 4527146.220000000 -17361 4519098.870000000 -8112 4514949.450000000 -13468 4499728.200000000 -18239 4497633.640000000 -10215 4494553.600000000 -6211 4492264.960000000 -836 4490945.100000000 -895 4489141.500000000 -19542 4488393.750000000 -4322 4487884.230000000 -2116 4486944.650000000 -553 4486075.480000000 -2515 4485188.260000000 -16286 4481470.470000000 -12271 4478224.950000000 -16570 4465818.000000000 -7995 4457574.660000000 -18396 4457229.600000000 -16331 4455735.480000000 -18157 4452196.630000000 -5271 4452040.010000000 -11622 4451244.840000000 -4052 4446397.340000000 -2864 4446008.380000000 -490 4442892.300000000 -19837 4434172.390000000 -4114 4433657.850000000 -11436 4433070.150000000 -6085 4431306.570000000 -9735 4430445.600000000 -17834 4416286.330000000 -8157 4416116.650000000 -18840 4414925.320000000 -13553 4412261.700000000 -12562 4411183.040000000 -14025 4403442.160000000 -17964 4400360.090000000 -636 4399863.840000000 -8390 4389024.330000000 -231 4387397.300000000 -9699 4385891.020000000 -10622 4384005.320000000 -14364 4383236.900000000 -10580 4381533.230000000 -10124 4369800.960000000 -10451 4368867.500000000 -4673 4367113.440000000 -11351 4362616.500000000 -4770 4362397.320000000 -12932 4362042.600000000 -10603 4357216.500000000 -19733 4348931.750000000 -4222 4348871.910000000 -17319 4347687.690000000 -3375 4346529.480000000 -14995 4338295.650000000 -7675 4337499.600000000 -15043 4333921.200000000 -4835 4332648.000000000 -4408 4332588.900000000 -5559 4330577.090000000 -7376 4328936.540000000 -18061 4328793.980000000 -2749 4328671.530000000 -6628 4328501.880000000 -5888 4323049.720000000 -18872 4322595.620000000 -5476 4319642.580000000 -1755 4318935.630000000 -10623 4315822.560000000 -18775 4314677.640000000 -3570 4312697.870000000 -11147 4310740.570000000 -6071 4307612.400000000 -10807 4306006.000000000 -9550 4299478.560000000 -657 4296794.190000000 -19669 4294640.900000000 -8532 4290651.600000000 -13469 4281715.620000000 -8809 4280778.800000000 -11301 4276847.950000000 -6147 4266879.920000000 -2612 4265962.350000000 -15699 4256118.720000000 -12300 4254409.110000000 -3494 4250810.600000000 -11040 4250030.200000000 -6190 4244046.800000000 -17616 4239937.500000000 -7271 4234407.000000000 -14048 4226977.440000000 -4456 4224684.980000000 -10012 4223841.210000000 -11175 4223704.140000000 -18675 4215406.860000000 -10792 4214898.570000000 -10806 4209678.320000000 -18749 4204787.000000000 -17410 4198025.280000000 -8032 4195430.000000000 -11094 4192304.940000000 -17582 4187341.440000000 -12246 4183230.950000000 -6640 4182968.800000000 -7346 4174707.600000000 -12747 4169865.810000000 -3869 4164957.440000000 -13106 4161902.080000000 -10547 4159541.360000000 -15289 4156205.760000000 -1679 4156156.640000000 -1126 4155593.080000000 -19106 4147439.520000000 -9705 4144024.200000000 -15324 4142518.560000000 -16544 4140375.720000000 -8812 4139322.810000000 -10772 4134101.640000000 -2800 4127150.080000000 -15549 4124704.640000000 -3607 4118697.570000000 -1980 4117633.720000000 -214 4113117.360000000 -19217 4104217.600000000 -2460 4098577.460000000 -19156 4093864.460000000 -18359 4092727.290000000 -12865 4092526.840000000 -14616 4092434.540000000 -908 4088856.200000000 -11791 4083804.970000000 -4157 4078345.600000000 -3857 4070872.870000000 -15114 4056112.500000000 -395 4052997.760000000 -17456 4051457.280000000 -10562 4050894.190000000 -10884 4050330.760000000 -12177 4049842.680000000 -15595 4040577.560000000 -15916 4036044.500000000 -7084 4035102.720000000 -4424 4034761.560000000 -10874 4031015.850000000 -4740 4030403.760000000 -16585 4030010.260000000 -18824 4028984.100000000 -14875 4028452.080000000 -13855 4024828.340000000 -10932 4024002.400000000 -9084 4021362.450000000 -14352 4018089.740000000 -18086 4015180.680000000 -9514 4013666.670000000 -15787 4013154.560000000 -714 4010249.440000000 -8811 4009588.900000000 -14386 4007210.880000000 -616 4004057.260000000 -7460 4003412.480000000 -866 4003182.540000000 -782 4001299.940000000 -8562 3999441.620000000 -1366 3994060.860000000 -2879 3993056.550000000 -16679 3992434.990000000 -17306 3990723.300000000 -13140 3982817.390000000 -17942 3980857.040000000 -6572 3977676.280000000 -3578 3977523.940000000 -15802 3969946.900000000 -336 3967938.380000000 -9807 3964469.600000000 -12104 3964273.400000000 -4271 3962359.280000000 -6702 3961657.440000000 -19763 3955582.750000000 -369 3953702.880000000 -4089 3953455.680000000 -2593 3946153.800000000 -590 3943841.160000000 -8325 3942118.750000000 -158 3941881.650000000 -12054 3938362.690000000 -18330 3938303.880000000 -5354 3936239.580000000 -8150 3925793.460000000 -8344 3921293.600000000 -6069 3921130.550000000 -4032 3920008.590000000 -17939 3917750.270000000 -7014 3914471.200000000 -2840 3913131.580000000 -1868 3912987.540000000 -10975 3911920.480000000 -5374 3910802.740000000 -11128 3908156.460000000 -18449 3907589.400000000 -11740 3907459.840000000 -2356 3907189.000000000 -5721 3901585.970000000 -4231 3900779.050000000 -4352 3899933.440000000 -432 3899836.440000000 -15321 3899516.580000000 -10296 3897015.140000000 -5647 3895088.160000000 -7386 3891916.510000000 -507 3891487.680000000 -3995 3887387.070000000 -4278 3882294.020000000 -18407 3880267.860000000 -6127 3879166.710000000 -145 3875277.240000000 -19269 3874685.760000000 -18257 3874454.890000000 -9068 3869767.740000000 -576 3860007.790000000 -4860 3852862.020000000 -18793 3849838.160000000 -15988 3847257.050000000 -6891 3846386.750000000 -3231 3846344.300000000 -15237 3845421.000000000 -9035 3844166.850000000 -7597 3838643.350000000 -16349 3837121.650000000 -2497 3827850.200000000 -3616 3827390.950000000 -11566 3826122.470000000 -18403 3822033.020000000 -2972 3821903.550000000 -812 3821523.720000000 -2043 3820561.360000000 -505 3818922.030000000 -8257 3815071.920000000 -6084 3814194.950000000 -11253 3813917.240000000 -366 3812257.880000000 -13632 3811601.320000000 -14298 3801412.420000000 -7092 3798729.480000000 -2058 3796109.040000000 -14820 3791195.860000000 -7157 3788690.820000000 -17211 3786030.170000000 -16644 3786019.250000000 -15693 3783662.190000000 -2627 3782394.600000000 -11231 3782077.600000000 -12696 3781761.660000000 -8705 3778077.000000000 -16052 3771577.040000000 -99 3760269.310000000 -2082 3757517.500000000 -872 3750005.340000000 -7126 3749138.920000000 -10302 3744475.250000000 -17122 3741012.980000000 -10080 3740107.100000000 -16021 3739611.200000000 -3074 3739224.960000000 -3142 3738811.020000000 -13213 3735116.250000000 -13442 3733132.140000000 -11542 3731000.120000000 -13732 3730444.900000000 -2608 3729372.400000000 -5 3725511.500000000 -19157 3723844.720000000 -18231 3721707.990000000 -8179 3714155.040000000 -12740 3708646.910000000 -11597 3706528.590000000 -13968 3702376.080000000 -6436 3687346.440000000 -9181 3687134.080000000 -564 3680200.800000000 -13464 3678406.200000000 -14084 3673790.380000000 -2755 3670593.690000000 -14284 3668640.800000000 -12178 3653392.480000000 -15730 3650258.300000000 -5560 3649569.590000000 -8594 3647140.560000000 -7032 3646439.540000000 -16846 3644843.100000000 -1530 3642838.080000000 -3978 3639712.050000000 -2897 3639442.320000000 -16625 3636527.540000000 -12029 3636339.720000000 -16830 3633448.570000000 -9597 3632662.110000000 -5533 3630338.670000000 -5181 3625965.930000000 -8131 3625738.620000000 -8560 3620761.260000000 -11860 3618746.250000000 -12008 3614604.400000000 -10737 3611990.640000000 -18208 3611596.100000000 -5119 3611038.200000000 -11958 3601654.650000000 -15124 3598278.200000000 -14058 3597490.020000000 -12270 3593912.100000000 -17793 3593318.950000000 -9385 3587327.840000000 -12814 3587083.840000000 -5304 3586230.610000000 -3631 3582841.650000000 -610 3581917.300000000 -19317 3580412.430000000 -128 3567004.560000000 -11616 3566154.800000000 -10176 3565392.150000000 -7349 3564110.640000000 -1712 3560408.430000000 -18860 3559340.600000000 -17617 3557516.000000000 -6443 3556296.960000000 -15408 3554814.560000000 -16350 3554388.630000000 -17436 3554105.130000000 -5740 3551324.680000000 -12181 3550218.540000000 -16895 3550119.300000000 -19995 3548839.700000000 -4968 3548306.870000000 -2257 3546692.290000000 -1825 3543198.780000000 -18989 3539038.080000000 -18727 3536081.400000000 -16165 3533789.840000000 -3249 3533709.870000000 -11731 3532875.000000000 -13032 3532415.790000000 -9377 3531582.080000000 -5883 3531479.000000000 -1211 3528833.400000000 -12065 3526948.100000000 -10866 3526146.660000000 -2073 3520131.300000000 -2378 3512186.200000000 -16860 3509693.070000000 -389 3507814.640000000 -15604 3505653.270000000 -11257 3502831.800000000 -1327 3502022.600000000 -16602 3501074.880000000 -1493 3498808.950000000 -8224 3498179.520000000 -622 3497158.360000000 -3072 3495958.720000000 -1478 3494880.480000000 -3125 3494169.900000000 -2052 3488438.080000000 -8476 3487191.280000000 -10735 3477740.760000000 -14860 3476235.840000000 -6586 3475745.100000000 -5130 3472024.500000000 -7181 3471306.300000000 -618 3467906.520000000 -15698 3464859.470000000 -17585 3462450.460000000 -2548 3456856.960000000 -2632 3456230.740000000 -2882 3453986.860000000 -12216 3452907.150000000 -4925 3452904.630000000 -9012 3442581.360000000 -6667 3430076.400000000 -17958 3424962.560000000 -6093 3424241.920000000 -10648 3417414.000000000 -1462 3413248.610000000 -2569 3412388.820000000 -18616 3409880.910000000 -7368 3408036.450000000 -3110 3407374.600000000 -10824 3406819.290000000 -11510 3404701.960000000 -4840 3397236.400000000 -4449 3396993.600000000 -1358 3396616.320000000 -3885 3395817.600000000 -13381 3391953.520000000 -1655 3383051.510000000 -282 3381785.420000000 -4928 3374270.480000000 -3199 3372488.800000000 -16086 3370710.650000000 -8612 3362922.500000000 -19597 3360764.000000000 -8867 3354400.110000000 -4098 3353574.280000000 -12617 3351499.050000000 -14365 3347296.000000000 -10443 3345493.100000000 -76 3342081.820000000 -11585 3341941.220000000 -4383 3338960.270000000 -13910 3335964.160000000 -8076 3332449.890000000 -16005 3332190.400000000 -2622 3329364.450000000 -12822 3321183.520000000 -17076 3320398.060000000 -5392 3320357.150000000 -18628 3319615.840000000 -13695 3318525.990000000 -10326 3318274.160000000 -9109 3317833.900000000 -1489 3317620.800000000 -3378 3315948.000000000 -7738 3312979.200000000 -1844 3312277.360000000 -19963 3307500.000000000 -2436 3306419.050000000 -886 3302180.700000000 -15475 3301693.500000000 -6327 3300680.780000000 -6050 3299460.200000000 -9876 3298410.050000000 -19586 3291131.250000000 -14349 3289862.520000000 -10993 3287980.570000000 -18784 3286752.120000000 -1800 3285466.240000000 -990 3284595.500000000 -3823 3281992.940000000 -15737 3279305.960000000 -19518 3276759.630000000 -9032 3272440.320000000 -7786 3271217.280000000 -8648 3271162.440000000 -5532 3270187.970000000 -15914 3268520.980000000 -16065 3265068.840000000 -11212 3264657.030000000 -13229 3262022.280000000 -15827 3260862.720000000 -1582 3260340.000000000 -3827 3260093.760000000 -3546 3259244.070000000 -15849 3258918.000000000 -14856 3258379.400000000 -2028 3255013.960000000 -6618 3254581.950000000 -17461 3252926.880000000 -13551 3241602.200000000 -19561 3239795.320000000 -2276 3236172.300000000 -14203 3234649.390000000 -7757 3231351.840000000 -122 3226213.880000000 -12954 3225943.000000000 -647 3224783.760000000 -12383 3223989.440000000 -3831 3223126.600000000 -16836 3222260.730000000 -4565 3221597.440000000 -19426 3218106.540000000 -17855 3217813.020000000 -5624 3207777.360000000 -8368 3203376.450000000 -9480 3200904.000000000 -11181 3199500.530000000 -8981 3197864.000000000 -16426 3195995.970000000 -1648 3195558.900000000 -14404 3192729.600000000 -17867 3188571.000000000 -18117 3183229.040000000 -14289 3182261.600000000 -53 3182256.000000000 -15546 3180180.040000000 -16245 3178277.460000000 -1597 3176247.480000000 -1653 3173456.640000000 -2845 3171619.610000000 -15906 3171187.540000000 -18304 3168571.500000000 -14068 3167367.600000000 -6837 3165012.480000000 -9446 3164446.520000000 -18889 3156140.960000000 -16587 3154210.200000000 -7705 3152977.380000000 -1120 3151591.170000000 -17665 3148848.000000000 -5311 3146721.860000000 -14157 3144707.320000000 -7996 3131351.040000000 -8663 3130526.320000000 -18271 3127800.960000000 -6446 3125685.960000000 -6972 3125007.060000000 -2572 3123186.830000000 -13536 3122527.540000000 -6196 3122172.480000000 -9338 3121262.400000000 -11992 3118647.550000000 -2580 3118284.370000000 -9098 3117494.100000000 -5118 3112661.960000000 -10184 3109293.400000000 -9932 3105818.240000000 -18545 3102273.320000000 -10963 3099314.500000000 -8405 3097121.120000000 -9037 3095195.000000000 -179 3091107.280000000 -1930 3090915.800000000 -17723 3090624.660000000 -4308 3089472.750000000 -8702 3080129.920000000 -18621 3079984.800000000 -4501 3079781.100000000 -3590 3079049.420000000 -18264 3078858.440000000 -15648 3078564.060000000 -5998 3073264.000000000 -16904 3072610.800000000 -3794 3071333.090000000 -3147 3068485.320000000 -17221 3068337.220000000 -4709 3067523.310000000 -18017 3066743.410000000 -15613 3063987.860000000 -16271 3057051.340000000 -13621 3054774.590000000 -12919 3054518.500000000 -12493 3050836.300000000 -15838 3050645.950000000 -3273 3048955.150000000 -8324 3046011.250000000 -13628 3045324.500000000 -5522 3044408.500000000 -2202 3043132.050000000 -19052 3042566.550000000 -5767 3041871.700000000 -17895 3036452.220000000 -12586 3036386.300000000 -12425 3035041.520000000 -13517 3034351.470000000 -2363 3033336.600000000 -15060 3032598.510000000 -6764 3032591.100000000 -340 3030522.000000000 -4723 3028910.250000000 -3566 3027858.610000000 -17796 3026838.960000000 -15384 3023792.640000000 -16336 3010813.560000000 -679 3010713.300000000 -7554 3010667.800000000 -14553 3009756.960000000 -8379 3009745.170000000 -15436 3007499.770000000 -12471 3003991.860000000 -18059 3003037.530000000 -8536 3000746.000000000 -19033 2999373.280000000 -18179 2996151.200000000 -10711 2996143.170000000 -17271 2994264.790000000 -13932 2989023.580000000 -3101 2987788.160000000 -14550 2977853.650000000 -3080 2977232.580000000 -14533 2976490.490000000 -14439 2975313.240000000 -9237 2973124.780000000 -1205 2971470.280000000 -12361 2963419.470000000 -429 2962631.880000000 -3970 2960418.450000000 -8403 2957698.450000000 -1098 2957514.000000000 -7932 2955046.140000000 -16266 2952298.380000000 -19386 2948854.480000000 -13147 2947037.910000000 -2720 2947011.080000000 -3840 2944219.350000000 -13482 2942474.880000000 -9436 2940396.210000000 -19779 2937105.960000000 -18032 2933224.380000000 -7743 2932733.770000000 -14620 2930766.890000000 -4606 2927832.590000000 -18076 2924134.830000000 -19276 2918176.200000000 -7483 2915918.950000000 -8575 2915132.640000000 -11052 2913140.880000000 -17251 2908345.800000000 -8788 2907935.930000000 -10960 2906511.140000000 -18782 2903643.780000000 -19988 2897461.530000000 -726 2896009.270000000 -19942 2894251.360000000 -10864 2892252.480000000 -17840 2891563.220000000 -18717 2888939.960000000 -12391 2886051.300000000 -18219 2885921.060000000 -15100 2883342.330000000 -2491 2880385.740000000 -12389 2879696.960000000 -3880 2877770.240000000 -18579 2874542.480000000 -13647 2873838.340000000 -15758 2873804.920000000 -12917 2873659.600000000 -18866 2873616.260000000 -13894 2872986.120000000 -15200 2872571.930000000 -9628 2872404.560000000 -8568 2871598.080000000 -8389 2870237.880000000 -5788 2867210.180000000 -19450 2863310.660000000 -9440 2863162.920000000 -16795 2860135.410000000 -19643 2858987.800000000 -1974 2856825.840000000 -14622 2852089.120000000 -6885 2851437.620000000 -12532 2848992.640000000 -1087 2847858.800000000 -5777 2846407.410000000 -5629 2846076.120000000 -6316 2840544.650000000 -12767 2840514.120000000 -12134 2840036.910000000 -14476 2839853.010000000 -803 2838388.160000000 -18964 2836942.440000000 -6020 2833459.200000000 -10401 2832688.740000000 -1323 2829964.500000000 -1151 2829662.440000000 -1458 2824034.430000000 -2271 2820756.530000000 -18740 2814140.800000000 -7348 2811730.950000000 -4281 2807190.520000000 -8043 2804706.240000000 -3843 2804217.960000000 -7813 2802350.880000000 -347 2802245.520000000 -745 2801725.100000000 -10388 2799170.580000000 -18100 2793358.500000000 -19043 2789013.800000000 -10644 2787797.010000000 -16170 2787402.800000000 -398 2782729.050000000 -9370 2780078.130000000 -14504 2780036.040000000 -1210 2778485.760000000 -13385 2777445.620000000 -3799 2775223.600000000 -11325 2769766.020000000 -3489 2769554.520000000 -17181 2769028.500000000 -6964 2766653.780000000 -7381 2764898.800000000 -6253 2764394.640000000 -5975 2760819.720000000 -11996 2760687.860000000 -7570 2758977.120000000 -4387 2757672.000000000 -9014 2755367.420000000 -9403 2748021.660000000 -11653 2739731.070000000 -17697 2739312.290000000 -958 2738032.000000000 -18816 2737140.000000000 -14104 2735008.640000000 -15966 2732250.200000000 -17912 2724160.950000000 -7089 2720170.040000000 -16032 2718976.180000000 -16891 2717293.320000000 -19579 2716909.860000000 -17470 2715048.840000000 -12408 2712556.520000000 -4763 2711800.900000000 -1138 2709709.810000000 -7363 2708414.400000000 -7877 2705439.450000000 -17532 2703698.680000000 -10512 2701235.920000000 -11957 2700133.220000000 -2455 2699593.880000000 -15119 2696860.800000000 -9868 2696801.520000000 -14172 2695307.480000000 -16120 2689337.820000000 -13958 2679025.280000000 -15169 2676686.040000000 -2648 2672232.000000000 -6164 2671317.320000000 -12701 2669216.400000000 -16382 2669034.540000000 -15588 2667212.100000000 -14830 2666758.150000000 -9119 2665812.240000000 -1622 2665206.500000000 -878 2664045.790000000 -13269 2662784.120000000 -619 2655417.630000000 -18386 2653795.020000000 -2501 2652260.400000000 -2310 2651631.090000000 -19420 2649395.610000000 -4895 2645152.270000000 -7553 2643682.070000000 -17814 2642781.440000000 -16097 2642500.000000000 -10995 2640811.160000000 -14895 2637733.720000000 -18546 2637026.710000000 -9875 2631358.800000000 -9591 2626899.540000000 -6001 2625893.760000000 -7739 2624573.280000000 -10431 2624379.540000000 -4544 2615313.750000000 -16387 2603195.760000000 -18375 2601407.830000000 -8395 2598728.440000000 -18853 2593356.360000000 -4900 2592813.150000000 -1302 2592197.760000000 -17032 2589806.400000000 -14292 2589749.560000000 -43 2587359.580000000 -5221 2587024.040000000 -397 2579751.460000000 -17890 2579674.240000000 -12157 2575510.480000000 -7340 2574645.830000000 -19368 2572618.950000000 -8848 2570819.150000000 -13789 2570243.260000000 -14596 2568234.240000000 -8408 2567434.410000000 -19726 2565750.420000000 -13964 2565579.120000000 -7740 2563027.500000000 -14768 2560392.600000000 -11734 2559062.220000000 -10294 2558257.970000000 -15032 2557926.220000000 -9127 2556379.800000000 -2181 2553175.000000000 -16653 2552229.680000000 -3866 2549994.790000000 -16814 2548710.760000000 -1866 2545838.400000000 -3512 2532626.800000000 -4145 2529786.150000000 -12120 2528298.720000000 -644 2528123.050000000 -15379 2525181.010000000 -6392 2524063.080000000 -2652 2521456.800000000 -3363 2519202.230000000 -19167 2517993.180000000 -16042 2516599.920000000 -2892 2511854.400000000 -5711 2509401.720000000 -14591 2506344.690000000 -6564 2506277.340000000 -1231 2505421.240000000 -5049 2502603.000000000 -14576 2501606.690000000 -10211 2500852.200000000 -293 2493168.480000000 -7371 2491134.650000000 -18154 2491047.200000000 -9494 2489825.520000000 -14836 2480432.400000000 -19471 2480403.750000000 -802 2478998.330000000 -12541 2477242.600000000 -15065 2473563.940000000 -15995 2472803.200000000 -9408 2471953.560000000 -9776 2470447.900000000 -17325 2468989.050000000 -3391 2468317.720000000 -16123 2467022.220000000 -18758 2463798.060000000 -407 2460304.470000000 -6840 2456170.780000000 -9995 2455155.360000000 -3877 2453696.650000000 -5817 2452493.130000000 -14122 2452226.220000000 -16699 2450273.980000000 -8921 2450116.480000000 -15103 2449861.200000000 -7637 2449628.720000000 -3076 2443927.380000000 -6648 2443248.950000000 -17116 2442263.720000000 -1645 2440838.400000000 -3181 2440017.600000000 -5966 2431558.080000000 -15882 2428947.300000000 -7529 2428381.280000000 -12836 2427897.330000000 -18052 2427637.760000000 -13616 2426638.500000000 -16615 2424775.080000000 -18147 2424412.680000000 -4586 2424123.900000000 -14403 2423141.960000000 -11606 2422794.310000000 -13526 2422212.800000000 -3677 2421404.460000000 -5553 2418506.210000000 -12109 2416514.170000000 -13118 2415931.800000000 -1563 2408855.400000000 -16591 2408045.390000000 -6411 2404918.530000000 -10272 2402834.480000000 -10597 2400247.680000000 -13700 2398035.860000000 -9548 2397147.900000000 -14963 2395781.090000000 -13325 2390637.580000000 -13864 2388067.880000000 -7450 2383447.710000000 -9275 2382868.400000000 -5829 2378037.920000000 -13437 2377806.540000000 -13594 2375046.300000000 -11442 2374591.080000000 -15619 2374052.380000000 -9063 2374035.840000000 -5990 2368686.500000000 -7811 2363829.260000000 -9525 2362974.530000000 -5597 2361031.840000000 -8963 2360774.000000000 -1709 2359839.290000000 -15814 2358656.640000000 -17613 2357519.040000000 -5022 2354550.450000000 -17740 2354242.830000000 -3388 2351042.260000000 -13773 2348739.120000000 -14467 2348665.040000000 -11544 2345324.450000000 -349 2344664.130000000 -10356 2340862.720000000 -18272 2338754.600000000 -4627 2337430.840000000 -327 2335298.460000000 -19846 2332224.730000000 -10814 2330319.600000000 -13102 2326122.750000000 -18867 2323972.000000000 -2824 2323315.080000000 -19117 2319911.100000000 -1906 2319757.600000000 -245 2319450.900000000 -17318 2317860.390000000 -3862 2316453.720000000 -8100 2313874.120000000 -2958 2312239.470000000 -10263 2308514.060000000 -13814 2304940.400000000 -9394 2303161.740000000 -18080 2299416.780000000 -1271 2289526.980000000 -3327 2278474.480000000 -8740 2278405.920000000 -8119 2276428.170000000 -3368 2274373.620000000 -7963 2272300.800000000 -2151 2270932.720000000 -16995 2270264.680000000 -9918 2269733.070000000 -503 2268535.250000000 -16692 2256484.500000000 -793 2254198.720000000 -16455 2252361.860000000 -6644 2249521.820000000 -17280 2249437.500000000 -6813 2248982.000000000 -4674 2246915.320000000 -16325 2244369.800000000 -182 2243290.000000000 -4626 2242474.350000000 -10860 2241291.600000000 -14034 2241220.800000000 -2476 2240855.200000000 -4253 2239985.640000000 -3211 2239871.020000000 -1290 2233313.000000000 -8479 2232189.040000000 -11895 2231607.000000000 -3487 2230171.620000000 -14870 2229915.370000000 -16328 2229483.960000000 -18585 2228215.500000000 -7638 2228208.080000000 -5436 2225672.280000000 -14594 2223005.070000000 -4532 2215711.020000000 -7586 2210562.510000000 -11870 2205182.820000000 -18487 2203653.600000000 -9179 2202720.520000000 -16500 2201185.310000000 -3679 2200592.700000000 -12803 2198295.000000000 -18056 2196741.900000000 -11396 2195645.640000000 -5087 2194120.720000000 -8067 2192048.640000000 -15357 2191646.580000000 -4491 2189713.500000000 -208 2189046.800000000 -10958 2188766.820000000 -9126 2188410.500000000 -15084 2184327.020000000 -18850 2183309.520000000 -3398 2180250.000000000 -16137 2177318.760000000 -211 2174808.960000000 -18422 2174381.000000000 -15840 2173510.400000000 -19553 2173079.770000000 -8221 2169992.160000000 -17000 2169611.160000000 -6755 2168505.150000000 -10817 2167710.680000000 -8327 2167650.600000000 -543 2167368.000000000 -4553 2163371.520000000 -15019 2162288.000000000 -334 2162178.480000000 -8516 2161479.040000000 -11349 2158941.880000000 -3902 2157027.860000000 -14731 2155302.240000000 -326 2153380.080000000 -11403 2151242.300000000 -11657 2150446.080000000 -9496 2149219.010000000 -8110 2149120.130000000 -5153 2148527.250000000 -884 2148324.980000000 -8637 2146185.100000000 -2364 2145790.720000000 -12386 2145001.470000000 -10133 2144903.960000000 -9895 2143324.800000000 -13755 2142539.400000000 -4327 2138501.400000000 -3369 2137408.760000000 -5815 2136985.000000000 -19357 2132657.280000000 -2675 2124158.720000000 -17869 2123991.720000000 -11702 2122132.990000000 -17257 2117850.640000000 -9952 2116686.320000000 -3881 2111457.150000000 -10951 2111185.580000000 -2128 2109702.300000000 -6699 2106578.400000000 -3155 2103636.640000000 -16649 2101956.200000000 -15257 2100297.750000000 -9978 2099566.560000000 -16810 2098301.440000000 -10653 2093388.700000000 -10476 2092766.480000000 -10883 2087495.280000000 -9704 2086967.610000000 -1119 2085182.840000000 -19139 2079788.340000000 -2144 2078391.140000000 -9135 2076377.800000000 -18548 2075584.320000000 -10545 2075230.350000000 -6220 2074341.720000000 -8616 2072887.650000000 -5230 2072161.740000000 -13916 2070504.720000000 -4299 2069922.960000000 -894 2069688.160000000 -17847 2063367.040000000 -18879 2061902.250000000 -13036 2061600.170000000 -10606 2060492.400000000 -9454 2060016.480000000 -118 2059808.860000000 -9601 2059715.760000000 -13769 2057668.080000000 -1987 2057289.270000000 -13863 2055368.000000000 -13562 2054754.240000000 -1840 2054183.920000000 -17995 2053221.900000000 -17389 2051128.200000000 -15168 2045987.490000000 -2139 2045365.400000000 -4024 2044243.100000000 -8964 2041648.850000000 -181 2040167.040000000 -7628 2039548.920000000 -3 2038846.090000000 -15553 2036958.910000000 -11355 2035405.600000000 -13006 2034991.200000000 -3091 2031393.510000000 -1281 2030628.480000000 -1408 2028621.660000000 -18211 2024538.670000000 -2287 2020754.320000000 -6228 2019198.820000000 -4362 2018495.250000000 -10873 2013280.320000000 -7383 2009581.920000000 -1386 2006544.260000000 -9820 2005815.760000000 -18134 2003409.730000000 -15727 2000654.500000000 -157 2000148.160000000 -19571 1999891.110000000 -17728 1997944.400000000 -5278 1996644.210000000 -17737 1994653.760000000 -10220 1989890.980000000 -1397 1984509.220000000 -6195 1983928.260000000 -4270 1983726.950000000 -16965 1983286.250000000 -1683 1980638.640000000 -13086 1978609.400000000 -7124 1974039.380000000 -5211 1973843.760000000 -6794 1973149.470000000 -257 1973035.440000000 -6995 1968281.550000000 -8447 1967292.700000000 -15873 1967257.890000000 -12862 1964014.130000000 -8295 1961467.080000000 -931 1958825.220000000 -6876 1957359.480000000 -1932 1954592.400000000 -1061 1952688.060000000 -18108 1951143.670000000 -5138 1950861.000000000 -12598 1950211.610000000 -10829 1943924.620000000 -11950 1941211.000000000 -12076 1939323.960000000 -2176 1938691.370000000 -6616 1937401.880000000 -5893 1934358.580000000 -976 1933066.800000000 -13173 1932557.520000000 -14947 1929229.980000000 -16857 1928814.800000000 -13403 1928702.880000000 -4819 1926969.680000000 -13127 1926929.830000000 -6871 1926787.680000000 -15465 1925145.090000000 -1131 1920005.500000000 -11845 1913576.400000000 -8364 1909122.200000000 -16588 1904272.370000000 -6759 1903906.290000000 -11586 1901895.650000000 -8145 1901787.660000000 -17333 1897297.200000000 -13290 1890633.750000000 -6499 1887621.000000000 -4881 1887535.920000000 -7147 1886710.200000000 -3883 1886567.780000000 -18911 1885597.120000000 -11336 1883573.600000000 -8653 1883275.760000000 -19476 1881492.480000000 -14799 1880543.400000000 -14491 1879219.920000000 -11815 1877434.340000000 -3173 1874302.100000000 -7161 1873023.450000000 -14631 1873015.300000000 -4247 1869912.960000000 -3568 1865824.400000000 -1500 1865451.030000000 -11833 1863665.230000000 -495 1860771.300000000 -6776 1855589.170000000 -11374 1855221.120000000 -5637 1853782.170000000 -3597 1852826.800000000 -981 1852083.600000000 -16076 1850349.690000000 -17597 1845420.950000000 -19609 1843185.480000000 -10997 1843072.020000000 -3403 1842975.000000000 -897 1842845.100000000 -16697 1840630.680000000 -17644 1840597.800000000 -6485 1838812.020000000 -5492 1836202.880000000 -12038 1835075.060000000 -9325 1832634.840000000 -10637 1832347.440000000 -11318 1830158.390000000 -4357 1828730.000000000 -18553 1826335.200000000 -12623 1825950.850000000 -961 1825869.600000000 -1677 1821816.900000000 -8211 1820432.520000000 -19719 1819333.550000000 -19663 1819074.350000000 -16296 1818353.770000000 -16527 1817834.420000000 -4964 1815400.020000000 -1769 1812929.200000000 -13126 1808799.960000000 -7854 1807608.060000000 -18380 1803641.220000000 -6584 1802346.980000000 -7665 1801765.350000000 -16553 1796146.780000000 -17761 1795095.720000000 -11179 1794890.300000000 -15171 1794148.720000000 -3018 1793183.880000000 -15741 1788612.000000000 -5331 1783901.350000000 -9860 1775071.260000000 -7984 1774302.750000000 -15354 1774270.770000000 -17884 1774212.440000000 -16257 1771869.710000000 -10696 1768645.200000000 -2104 1767902.640000000 -14465 1764946.400000000 -10089 1764692.320000000 -6719 1762699.540000000 -3648 1760594.420000000 -7241 1759913.590000000 -11122 1757430.040000000 -17019 1752560.650000000 -13877 1744271.100000000 -15325 1743826.260000000 -17860 1739870.440000000 -2236 1739795.800000000 -4436 1738760.320000000 -7701 1738670.400000000 -8147 1736855.160000000 -6676 1736341.440000000 -19505 1735413.430000000 -9885 1731366.260000000 -2112 1725934.080000000 -5330 1722196.980000000 -3561 1720377.960000000 -10104 1714419.160000000 -16362 1712457.380000000 -15573 1712365.440000000 -15006 1711381.350000000 -14629 1709942.050000000 -9612 1709528.380000000 -19910 1709211.150000000 -13145 1708907.460000000 -11494 1707973.680000000 -15895 1706999.450000000 -8239 1705479.100000000 -2403 1705331.100000000 -19436 1702706.000000000 -3476 1702335.800000000 -6828 1702292.080000000 -771 1701589.500000000 -8448 1700312.440000000 -3755 1699047.030000000 -13895 1698679.030000000 -9785 1698056.370000000 -6180 1695571.530000000 -532 1694356.150000000 -6741 1692552.420000000 -19964 1692367.640000000 -3747 1691244.600000000 -3253 1690719.420000000 -16119 1688339.250000000 -7113 1681911.000000000 -12368 1681219.800000000 -16378 1679705.600000000 -1393 1675545.350000000 -11119 1675453.440000000 -4469 1674023.490000000 -6955 1672618.900000000 -11579 1672345.320000000 -19898 1671781.700000000 -15351 1659204.300000000 -6133 1658215.460000000 -9110 1658054.680000000 -2979 1656016.740000000 -18764 1653708.480000000 -8995 1653627.580000000 -13096 1651408.670000000 -15062 1650548.020000000 -7924 1650202.400000000 -10076 1647970.240000000 -15859 1646036.280000000 -17932 1642640.660000000 -19694 1642089.500000000 -13827 1642001.310000000 -17963 1639689.000000000 -10698 1635848.260000000 -18003 1633530.780000000 -8416 1633366.770000000 -476 1631154.060000000 -2806 1630782.800000000 -12129 1628615.470000000 -11215 1626624.700000000 -14061 1624933.440000000 -5956 1623586.100000000 -9043 1622670.400000000 -13287 1621980.360000000 -11410 1621420.900000000 -13990 1621268.200000000 -12952 1619215.180000000 -15181 1619088.680000000 -9784 1618120.530000000 -10733 1616168.880000000 -16054 1614531.230000000 -5864 1614397.830000000 -1875 1611927.000000000 -17381 1611664.800000000 -14562 1607467.920000000 -575 1605941.730000000 -2005 1605591.720000000 -4332 1605448.830000000 -4653 1602596.300000000 -15403 1601830.440000000 -17430 1599681.420000000 -4798 1593630.500000000 -12991 1593321.520000000 -15653 1593138.660000000 -10066 1593049.060000000 -8892 1592100.900000000 -6708 1590159.120000000 -9825 1589403.920000000 -8271 1588475.410000000 -17084 1584280.880000000 -4003 1583631.000000000 -869 1582643.160000000 -16400 1582313.200000000 -19088 1581708.560000000 -6581 1581346.800000000 -9481 1581048.600000000 -6092 1580846.490000000 -3624 1578777.300000000 -6503 1578507.780000000 -14557 1578280.960000000 -2428 1577543.920000000 -15513 1573560.210000000 -4641 1573363.540000000 -10152 1570213.600000000 -5932 1566902.520000000 -7482 1561323.500000000 -13745 1558358.340000000 -2251 1558274.700000000 -9845 1558068.120000000 -7603 1557388.200000000 -1809 1553837.200000000 -18128 1547643.360000000 -8086 1543199.040000000 -14948 1541721.570000000 -16725 1540948.500000000 -2999 1540317.660000000 -8861 1540008.470000000 -1964 1538815.250000000 -19374 1537884.780000000 -15428 1535994.360000000 -7449 1534782.480000000 -16884 1534509.160000000 -10271 1534397.340000000 -11782 1529963.220000000 -8184 1529750.700000000 -4560 1527433.240000000 -4616 1525374.460000000 -3814 1524077.040000000 -17265 1523932.080000000 -16520 1522906.280000000 -10475 1518705.060000000 -5094 1517317.830000000 -8626 1515142.070000000 -19895 1512286.680000000 -19933 1506235.360000000 -6854 1505626.000000000 -13995 1505562.180000000 -7102 1504945.670000000 -9079 1501237.200000000 -18329 1500146.900000000 -3742 1496990.770000000 -12395 1496904.430000000 -12214 1496489.400000000 -12298 1495554.300000000 -4978 1495389.500000000 -2927 1494280.100000000 -2119 1494151.140000000 -15143 1492039.750000000 -14548 1487406.600000000 -840 1486128.980000000 -5902 1486097.280000000 -10614 1482144.720000000 -5895 1481356.800000000 -15958 1480951.600000000 -11408 1479948.960000000 -8407 1474236.000000000 -6243 1471007.850000000 -10389 1469004.460000000 -13871 1468938.640000000 -19811 1464597.090000000 -10495 1464290.490000000 -4389 1463010.830000000 -1311 1461703.360000000 -17874 1459408.880000000 -6597 1458761.870000000 -19211 1456741.630000000 -12879 1456178.240000000 -8840 1455731.460000000 -14755 1454890.600000000 -16957 1454465.960000000 -9257 1454388.760000000 -5193 1454011.320000000 -6884 1452474.600000000 -19948 1452024.000000000 -15076 1448395.000000000 -16016 1447557.450000000 -11693 1445839.680000000 -6975 1440516.960000000 -4290 1439768.460000000 -18900 1438722.100000000 -14383 1438477.920000000 -15098 1435941.780000000 -9322 1435282.800000000 -458 1433040.450000000 -10042 1432906.350000000 -5052 1431900.900000000 -6600 1431116.550000000 -3630 1428665.040000000 -9636 1428193.840000000 -16511 1427308.740000000 -4045 1427248.350000000 -19562 1426348.820000000 -8814 1425690.090000000 -2616 1425178.040000000 -4587 1425109.400000000 -148 1424237.370000000 -2712 1423780.260000000 -10863 1423386.160000000 -16096 1421942.090000000 -18936 1421938.650000000 -18327 1419872.920000000 -11620 1419050.100000000 -3740 1418609.850000000 -3457 1418603.500000000 -1185 1417637.470000000 -8178 1417357.260000000 -17791 1413293.130000000 -13608 1411323.120000000 -17849 1409613.500000000 -6814 1406228.400000000 -14022 1406138.040000000 -14231 1403771.520000000 -19546 1402854.600000000 -19619 1402389.160000000 -5609 1402302.540000000 -5342 1401567.590000000 -3084 1401096.100000000 -5708 1400334.900000000 -17998 1399862.450000000 -19850 1397630.330000000 -14004 1395443.100000000 -13071 1394653.240000000 -2797 1393747.580000000 -2866 1392947.250000000 -19809 1389067.680000000 -13600 1380865.800000000 -13614 1380654.360000000 -5884 1380319.740000000 -9404 1378623.660000000 -10656 1376954.320000000 -12324 1376502.400000000 -7325 1375030.430000000 -13295 1373987.340000000 -11864 1373555.680000000 -6987 1373481.510000000 -8386 1371854.410000000 -10916 1370374.320000000 -12867 1369058.110000000 -14668 1369040.340000000 -13383 1367342.300000000 -18572 1366953.960000000 -1152 1366861.380000000 -6015 1366452.180000000 -3344 1366185.150000000 -7889 1365521.920000000 -13345 1364088.880000000 -6276 1363421.620000000 -8069 1361824.200000000 -17509 1360892.490000000 -15137 1358678.070000000 -17163 1357391.520000000 -4704 1356692.400000000 -8609 1356578.190000000 -12644 1356088.140000000 -17141 1356022.380000000 -11805 1354826.780000000 -6386 1354187.220000000 -3004 1352173.440000000 -8634 1350211.800000000 -4399 1349881.200000000 -10362 1349411.340000000 -1572 1348835.200000000 -7359 1348224.100000000 -11884 1346696.820000000 -11671 1346424.150000000 -5350 1346359.280000000 -3119 1345996.480000000 -5307 1345356.000000000 -16117 1345045.120000000 -8715 1342665.720000000 -5398 1341179.280000000 -7627 1338820.560000000 -8457 1337714.680000000 -4958 1334732.710000000 -84 1334146.710000000 -6932 1333235.360000000 -757 1332921.070000000 -4076 1332441.000000000 -1751 1329112.320000000 -15701 1327052.160000000 -4119 1326549.900000000 -1562 1325604.280000000 -8741 1325517.600000000 -1135 1325422.710000000 -1002 1323418.650000000 -5832 1323085.710000000 -5368 1322793.960000000 -5382 1322628.840000000 -5616 1319082.260000000 -2832 1318691.950000000 -3895 1317858.440000000 -8629 1317756.510000000 -5709 1317058.680000000 -18383 1316451.050000000 -15797 1314806.640000000 -1900 1313660.400000000 -13882 1310455.860000000 -6785 1309877.800000000 -14855 1309280.760000000 -7761 1308602.240000000 -14268 1306810.400000000 -6257 1306056.960000000 -19002 1305509.520000000 -5095 1303729.020000000 -10320 1301657.580000000 -7826 1299561.680000000 -13359 1298717.140000000 -7436 1298127.360000000 -5644 1295055.770000000 -11327 1290526.410000000 -5277 1289329.650000000 -15932 1286235.840000000 -14322 1284809.360000000 -144 1284270.120000000 -3043 1281162.790000000 -16788 1280955.340000000 -17136 1280443.120000000 -12560 1279117.950000000 -13833 1278834.750000000 -5414 1277893.260000000 -12582 1277592.320000000 -4644 1277535.000000000 -14032 1277077.880000000 -18325 1271719.680000000 -7072 1271228.480000000 -16868 1267469.420000000 -8137 1267425.810000000 -5976 1266206.850000000 -14125 1265569.050000000 -13299 1265287.550000000 -18376 1264249.300000000 -6157 1261759.920000000 -5002 1261669.640000000 -13368 1260918.600000000 -15589 1260059.760000000 -2149 1258981.440000000 -9639 1256283.380000000 -11689 1256027.920000000 -9083 1245924.240000000 -16231 1242625.650000000 -5084 1242385.280000000 -11634 1240760.180000000 -15617 1239731.250000000 -9865 1237181.620000000 -14212 1236365.520000000 -10325 1235223.360000000 -19582 1235105.760000000 -740 1234746.810000000 -19231 1233623.100000000 -16840 1233063.850000000 -5703 1231744.330000000 -5761 1229435.200000000 -15630 1226611.620000000 -10408 1224698.400000000 -9177 1221942.510000000 -13389 1221666.750000000 -6104 1221577.920000000 -9673 1218826.640000000 -2707 1217124.480000000 -18672 1214208.800000000 -5112 1209590.200000000 -6264 1208318.500000000 -18496 1207881.750000000 -10971 1207183.520000000 -19059 1206729.900000000 -431 1205938.440000000 -3821 1201192.750000000 -826 1200454.620000000 -3317 1200440.900000000 -19689 1198899.520000000 -19641 1198797.990000000 -6379 1197195.500000000 -814 1194417.400000000 -18643 1194000.780000000 -11865 1193965.760000000 -12393 1193896.800000000 -9218 1193660.580000000 -8674 1191881.320000000 -8582 1191804.020000000 -13084 1191508.000000000 -18844 1190239.960000000 -16061 1189935.000000000 -6134 1185550.800000000 -8628 1183245.600000000 -8884 1181547.480000000 -7697 1181032.500000000 -9044 1180922.600000000 -13257 1180158.570000000 -8066 1178808.120000000 -5876 1177376.800000000 -14694 1177059.310000000 -16062 1175391.000000000 -9104 1175178.900000000 -11600 1175091.060000000 -10337 1172684.920000000 -19188 1172349.780000000 -8833 1171372.930000000 -6895 1170602.070000000 -14100 1168878.400000000 -13538 1168554.280000000 -3408 1166645.160000000 -1860 1165673.680000000 -13436 1164278.700000000 -19325 1162733.700000000 -7403 1161982.000000000 -4882 1161404.810000000 -13105 1161320.580000000 -17880 1161256.020000000 -19284 1160927.600000000 -13476 1159035.150000000 -18913 1158208.300000000 -18523 1158135.000000000 -12508 1157538.450000000 -9090 1156362.640000000 -17653 1154338.080000000 -3926 1152652.520000000 -10183 1148324.570000000 -7556 1146268.140000000 -16436 1142656.470000000 -4741 1141614.000000000 -15651 1141497.930000000 -3183 1140081.360000000 -9532 1139902.500000000 -16403 1139306.370000000 -2368 1137421.160000000 -3889 1136395.500000000 -2885 1135838.140000000 -7851 1135110.760000000 -16234 1135017.240000000 -12746 1134531.040000000 -2647 1132941.120000000 -5373 1132158.010000000 -10340 1132004.240000000 -8873 1131949.280000000 -1132 1131338.880000000 -15594 1131328.620000000 -4376 1130282.200000000 -240 1126682.480000000 -2231 1124447.150000000 -929 1121383.920000000 -11599 1119307.270000000 -3765 1119093.500000000 -17635 1118420.160000000 -7119 1118285.080000000 -15121 1117715.340000000 -11858 1116963.540000000 -16963 1116929.450000000 -16356 1113648.980000000 -6924 1112198.400000000 -16223 1111257.000000000 -18091 1110043.020000000 -12628 1108954.800000000 -16043 1108831.050000000 -9402 1108290.480000000 -708 1107084.000000000 -4078 1105993.960000000 -17593 1104713.400000000 -12776 1104362.590000000 -7583 1102813.530000000 -14619 1102675.800000000 -8842 1100110.260000000 -4196 1099726.550000000 -2019 1098178.640000000 -6863 1097246.360000000 -6489 1096503.070000000 -2459 1094813.040000000 -11964 1094485.020000000 -3236 1093969.800000000 -17647 1093809.150000000 -17648 1093114.620000000 -119 1092687.480000000 -9626 1092080.000000000 -9124 1091569.680000000 -13175 1089851.760000000 -2532 1088706.350000000 -16083 1088295.390000000 -8874 1086011.340000000 -12872 1082970.300000000 -19821 1082520.840000000 -4800 1080389.700000000 -18696 1079685.360000000 -19545 1079184.330000000 -13120 1077742.280000000 -10588 1076203.830000000 -17696 1075092.720000000 -14651 1073222.230000000 -903 1071146.760000000 -5858 1070259.480000000 -8302 1069504.800000000 -18728 1069225.510000000 -18026 1068569.000000000 -19383 1066907.580000000 -18690 1065930.900000000 -5924 1065143.120000000 -4880 1065011.750000000 -12439 1064381.190000000 -16529 1062371.700000000 -19653 1057683.560000000 -3136 1056810.440000000 -18932 1056193.650000000 -2124 1054160.520000000 -16851 1052646.840000000 -10123 1051624.000000000 -5618 1048447.930000000 -19851 1045187.850000000 -16278 1044808.380000000 -11479 1044276.220000000 -13263 1042046.200000000 -6041 1041123.380000000 -7193 1040455.320000000 -19408 1039430.010000000 -11260 1036828.520000000 -5179 1035633.440000000 -1331 1034398.000000000 -7706 1034249.400000000 -8436 1033549.350000000 -1801 1031886.000000000 -4170 1031642.900000000 -11827 1031139.390000000 -17114 1027985.880000000 -18278 1026583.110000000 -1995 1025165.680000000 -7667 1022980.150000000 -6559 1021635.450000000 -17488 1021612.130000000 -16059 1019781.190000000 -7633 1018782.570000000 -10032 1016809.500000000 -2899 1016438.760000000 -14628 1016033.200000000 -10126 1015846.780000000 -3884 1014413.500000000 -16913 1013604.400000000 -18644 1010288.100000000 -19870 1007919.360000000 -18564 1007416.200000000 -10179 1004920.000000000 -883 1004650.680000000 -3627 1004461.040000000 -18408 1002285.180000000 -8785 1001660.060000000 -18849 1000916.400000000 -12440 999894.160000000 -18516 999274.600000000 -5755 999117.990000000 -9739 998721.870000000 -16958 998513.050000000 -15352 996377.100000000 -9299 994737.100000000 -11553 994664.260000000 -2032 993403.950000000 -19125 992934.150000000 -7538 989526.150000000 -18943 988533.000000000 -8665 987149.000000000 -16459 986031.930000000 -368 985751.040000000 -4136 984763.650000000 -7827 984693.040000000 -3333 982245.380000000 -17809 980965.400000000 -932 980620.680000000 -16678 980499.910000000 -9964 977563.800000000 -14358 976236.280000000 -19715 974700.160000000 -4639 974392.240000000 -5628 974254.050000000 -13143 973719.030000000 -11088 972929.960000000 -110 968796.010000000 -16030 968498.400000000 -12092 967595.160000000 -19616 966431.340000000 -1847 965957.160000000 -13477 965756.250000000 -9186 962981.340000000 -6622 962153.520000000 -13043 962126.310000000 -6368 960535.160000000 -6557 960340.500000000 -11474 959540.400000000 -9755 959381.110000000 -4778 959120.160000000 -12616 956904.300000000 -18851 956179.560000000 -9107 954695.520000000 -271 954298.650000000 -19119 949928.750000000 -10375 947371.060000000 -7643 942287.220000000 -18582 941870.280000000 -14606 939281.150000000 -8483 939067.940000000 -3651 937217.960000000 -17607 936620.160000000 -5585 935459.980000000 -1630 932812.120000000 -7500 931442.750000000 -16824 929731.440000000 -13422 927685.440000000 -13801 926400.900000000 -18995 926105.520000000 -2334 924636.130000000 -13206 924175.170000000 -963 922402.660000000 -14908 922366.470000000 -4325 921271.440000000 -4886 920831.000000000 -7253 920276.000000000 -3751 919262.680000000 -16282 918933.000000000 -6302 918429.410000000 -13655 918324.000000000 -12316 914770.220000000 -3179 911620.500000000 -13438 909114.260000000 -11488 909044.620000000 -11361 908145.200000000 -10477 901784.580000000 -7365 900147.170000000 -19277 899900.440000000 -226 899761.380000000 -16287 899454.580000000 -4390 898878.630000000 -1550 898776.000000000 -9644 898445.100000000 -17330 897248.310000000 -12650 895000.480000000 -658 894649.920000000 -15284 892571.400000000 -19024 891979.200000000 -10377 891389.460000000 -7476 890710.190000000 -16253 890379.820000000 -4995 890107.440000000 -11770 885148.580000000 -18360 884180.880000000 -686 883878.600000000 -19181 882953.940000000 -18204 882570.920000000 -8644 881690.810000000 -4417 880993.500000000 -2633 880910.370000000 -6532 876322.500000000 -8081 871985.600000000 -3963 871550.900000000 -16327 871208.520000000 -9659 869041.500000000 -10599 868484.650000000 -3532 866357.380000000 -378 865911.540000000 -15502 865112.380000000 -17131 864977.080000000 -11367 862828.370000000 -2157 862828.160000000 -5603 861152.100000000 -10964 854847.210000000 -19888 850844.770000000 -10349 848868.440000000 -10257 846497.000000000 -9349 844048.480000000 -9609 843101.020000000 -14240 842825.540000000 -12333 842271.880000000 -7894 840906.680000000 -11625 840127.080000000 -255 838524.180000000 -13492 836492.800000000 -16386 831304.500000000 -13550 830651.940000000 -12610 829164.600000000 -1335 829070.500000000 -648 824028.100000000 -18645 823187.200000000 -6452 822401.700000000 -8774 821486.360000000 -2454 821166.880000000 -17060 817472.280000000 -18248 817428.480000000 -7272 816893.220000000 -16363 814467.030000000 -11972 810825.980000000 -4386 810015.900000000 -4841 807959.030000000 -9157 807780.540000000 -7444 803362.540000000 -19253 801097.750000000 -17059 801055.180000000 -7518 800781.600000000 -8070 800485.840000000 -3779 794856.700000000 -9303 794784.960000000 -3819 794693.520000000 -11473 794457.600000000 -2539 793908.970000000 -18664 792838.160000000 -18789 790057.450000000 -16474 787600.600000000 -18115 787415.900000000 -2075 784851.840000000 -19375 783701.500000000 -19202 782566.100000000 -17004 782347.680000000 -14088 781780.600000000 -2823 780046.650000000 -9445 779740.060000000 -2370 778730.550000000 -2345 778411.800000000 -5643 777348.600000000 -11521 776828.360000000 -19830 776390.300000000 -2767 775894.470000000 -6328 775090.580000000 -6119 775083.000000000 -15458 774661.780000000 -13840 774443.000000000 -14648 771094.720000000 -5677 768204.360000000 -12553 767871.260000000 -2677 767754.050000000 -14120 764268.750000000 -5349 764223.460000000 -8052 763781.000000000 -12912 763767.200000000 -15591 763521.280000000 -11386 762604.920000000 -15365 762590.400000000 -5586 760490.640000000 -5319 760122.220000000 -14419 759407.000000000 -10897 758587.680000000 -8013 756712.460000000 -18467 756137.400000000 -10339 755974.380000000 -1051 755393.580000000 -18808 754670.070000000 -10672 752363.080000000 -10752 751872.000000000 -18611 751215.300000000 -868 748493.060000000 -16084 745210.100000000 -19340 744998.040000000 -8876 744681.900000000 -4147 743885.810000000 -4122 743555.200000000 -8852 741043.030000000 -14893 740065.500000000 -11918 739955.900000000 -3502 739633.110000000 -6227 739209.500000000 -9187 737922.400000000 -18280 737507.190000000 -8899 736686.930000000 -10253 736572.340000000 -4140 734955.420000000 -19878 732123.810000000 -6739 732000.010000000 -7261 731959.620000000 -17110 731579.160000000 -17194 730970.350000000 -12595 730734.380000000 -9850 728298.600000000 -18996 725894.760000000 -6997 721063.200000000 -5669 720956.700000000 -2184 720528.000000000 -16987 718963.000000000 -15608 718666.560000000 -10553 718099.200000000 -3134 717103.300000000 -7070 715509.600000000 -17645 714081.640000000 -12121 713814.610000000 -1279 710088.750000000 -11779 709146.250000000 -6621 708524.640000000 -16597 708244.950000000 -7146 706965.750000000 -17518 706355.430000000 -4154 704086.880000000 -16090 702130.200000000 -16610 700366.380000000 -13823 700025.100000000 -11470 699696.340000000 -8269 699644.410000000 -18368 697225.980000000 -16359 696749.040000000 -4562 696027.810000000 -11530 695781.240000000 -10888 695332.440000000 -7840 694064.780000000 -18285 692323.100000000 -17821 691327.800000000 -4509 690905.520000000 -1107 687277.800000000 -17600 687171.000000000 -7326 686379.120000000 -4649 685479.200000000 -11106 684800.000000000 -10586 683696.230000000 -16850 682672.320000000 -11648 682541.480000000 -11010 682487.840000000 -13821 682057.670000000 -18140 681645.480000000 -16944 681058.490000000 -16476 680013.450000000 -11084 677803.740000000 -15616 675492.480000000 -15052 675143.370000000 -18604 675065.590000000 -16157 672871.840000000 -17586 672865.920000000 -3586 670977.060000000 -17383 670781.640000000 -11211 667600.920000000 -9553 667158.470000000 -10805 666257.560000000 -18159 664172.340000000 -7408 661874.240000000 -8084 660284.420000000 -17043 659951.760000000 -847 659530.180000000 -13797 657922.800000000 -5857 657834.090000000 -11886 657353.690000000 -19740 656545.070000000 -12043 655994.500000000 -19263 654466.450000000 -18592 649935.000000000 -7974 649679.320000000 -6607 649092.210000000 -634 648192.940000000 -16551 646627.520000000 -16211 645862.800000000 -4095 643986.240000000 -1114 639657.040000000 -5383 638309.280000000 -8298 637017.600000000 -13123 632410.160000000 -3052 631869.150000000 -8091 630857.500000000 -9922 630642.880000000 -4791 630131.330000000 -1436 629369.860000000 -1426 628241.250000000 -17295 627959.180000000 -3863 627598.280000000 -97 626592.120000000 -15892 624640.410000000 -17350 624434.910000000 -11771 623709.360000000 -3395 623076.840000000 -552 622856.590000000 -8121 620644.500000000 -14380 620623.360000000 -10480 618647.200000000 -1792 617391.320000000 -7962 615206.660000000 -5446 615127.730000000 -7224 615007.800000000 -14166 614256.840000000 -16817 614072.600000000 -5298 613673.100000000 -15562 612427.800000000 -16197 611744.400000000 -7835 611155.540000000 -7187 611130.960000000 -18377 609733.310000000 -7644 609672.000000000 -645 607985.840000000 -12539 607267.100000000 -4568 606708.900000000 -19353 605121.650000000 -3157 605048.800000000 -12790 604517.760000000 -18459 604378.110000000 -13685 603255.060000000 -9817 603241.350000000 -11126 602064.540000000 -19538 601517.850000000 -17179 600665.600000000 -10134 600237.120000000 -4875 599774.700000000 -6923 599427.660000000 -17902 598573.300000000 -16489 596732.640000000 -1883 596048.730000000 -15960 596045.790000000 -12366 595898.010000000 -15310 594751.850000000 -3860 593622.120000000 -16799 592959.150000000 -10628 592922.980000000 -7884 592282.080000000 -13025 591341.650000000 -10052 589303.440000000 -2386 586754.550000000 -5466 586707.720000000 -5327 584317.500000000 -302 582503.760000000 -14684 581500.790000000 -3904 580408.080000000 -17550 580311.930000000 -10115 578872.020000000 -10436 578429.000000000 -14227 577834.640000000 -15597 577684.500000000 -15530 576776.100000000 -4495 575799.120000000 -19015 573844.050000000 -19446 573115.290000000 -521 572606.300000000 -8000 571204.260000000 -11075 571052.580000000 -5369 569455.600000000 -19359 567754.010000000 -15147 567201.420000000 -11623 566420.400000000 -18540 565692.190000000 -17147 565595.200000000 -4385 565432.860000000 -10102 564403.200000000 -16308 563740.510000000 -14858 563180.200000000 -9222 561328.110000000 -15434 561081.780000000 -18353 561052.800000000 -13323 559636.050000000 -4134 559387.860000000 -16121 558160.470000000 -4903 556776.000000000 -17776 554339.180000000 -16408 552944.700000000 -19746 552381.440000000 -12885 551994.070000000 -1615 550834.350000000 -10235 550447.740000000 -7755 549525.600000000 -5488 547176.060000000 -2884 545432.910000000 -17377 543375.960000000 -18653 542456.460000000 -4201 542010.070000000 -15811 540814.400000000 -6413 539882.490000000 -4306 538725.120000000 -2780 538546.330000000 -15945 536488.400000000 -14461 535327.960000000 -8966 535307.080000000 -4878 534848.320000000 -2131 534687.160000000 -3930 533828.240000000 -2408 533672.940000000 -16102 532910.750000000 -18228 532842.450000000 -11271 532651.240000000 -682 531371.700000000 -14993 530208.300000000 -19179 529993.600000000 -8218 528876.800000000 -13558 528740.550000000 -3458 528294.910000000 -3300 526425.200000000 -13731 525692.050000000 -7132 525692.010000000 -8960 524338.560000000 -711 523794.180000000 -11914 523793.180000000 -19404 522622.620000000 -19912 522064.920000000 -17356 519632.120000000 -15231 518574.290000000 -3622 518021.780000000 -7440 515730.810000000 -19648 515645.520000000 -13859 514836.400000000 -17954 514679.550000000 -3996 514030.560000000 -1823 513844.380000000 -16740 512589.200000000 -3642 510694.380000000 -3133 508520.320000000 -10840 507788.990000000 -13271 506252.700000000 -19845 506037.700000000 -1596 505225.820000000 -6877 504480.480000000 -16648 502701.900000000 -6072 501742.520000000 -14384 501514.090000000 -7275 501492.540000000 -10816 500849.700000000 -19783 499605.400000000 -13727 499594.550000000 -19379 498554.140000000 -13619 497012.620000000 -15649 495023.750000000 -827 494884.180000000 -8037 494712.990000000 -4817 491132.270000000 -12032 491090.020000000 -14989 490884.310000000 -15267 490589.580000000 -10216 489859.590000000 -19464 489076.160000000 -5076 488878.770000000 -18562 486103.560000000 -19151 484434.150000000 -13687 484196.360000000 -14532 484177.520000000 -12563 483576.720000000 -9217 483259.200000000 -6348 481950.560000000 -15248 481809.240000000 -19884 479916.060000000 -2929 479758.020000000 -16431 479482.380000000 -18627 479153.920000000 -11995 479067.840000000 -10147 478475.860000000 -18191 477161.250000000 -10354 476558.640000000 -16871 476466.900000000 -9147 475695.830000000 -19615 475611.120000000 -7964 475007.730000000 -9296 474259.590000000 -9534 473728.500000000 -5963 473472.780000000 -8339 472930.180000000 -2208 469865.250000000 -1338 469668.600000000 -9629 469466.790000000 -8615 469072.310000000 -1589 468032.320000000 -1142 467469.620000000 -11398 466612.170000000 -14524 465466.710000000 -18343 465072.510000000 -12174 462715.600000000 -5113 461798.420000000 -3964 460889.710000000 -5072 460641.750000000 -19532 457296.900000000 -16368 457068.500000000 -13327 455568.750000000 -3498 455507.800000000 -14515 454234.200000000 -2376 453831.460000000 -13842 453709.800000000 -17817 453362.070000000 -9581 453321.000000000 -6874 453007.300000000 -6152 452603.800000000 -2528 451368.750000000 -12445 451065.420000000 -14286 450652.800000000 -8046 449859.690000000 -13743 449663.110000000 -16881 447039.650000000 -4624 446875.400000000 -17039 446598.240000000 -9558 446209.920000000 -14333 446137.120000000 -1147 445357.160000000 -4963 444298.020000000 -16764 442600.660000000 -3130 440642.370000000 -12291 438361.530000000 -10616 438327.650000000 -1891 438161.500000000 -1869 435965.600000000 -7791 433569.140000000 -7862 433004.810000000 -7882 432844.100000000 -13856 432348.940000000 -17567 432067.600000000 -3047 431188.280000000 -15462 430794.520000000 -18940 430547.440000000 -5743 430353.200000000 -6901 430326.950000000 -394 430080.020000000 -5574 429849.120000000 -13669 429391.050000000 -19695 429200.280000000 -633 428209.600000000 -13052 427656.160000000 -4108 427362.540000000 -18958 425107.950000000 -7648 422904.240000000 -8602 422748.260000000 -2905 422636.000000000 -10627 421632.810000000 -15023 419740.520000000 -13288 419592.390000000 -10293 419449.950000000 -15791 419040.720000000 -7795 419012.490000000 -2211 418460.160000000 -12106 417856.400000000 -17380 417316.700000000 -16991 417144.000000000 -13076 416336.800000000 -5506 415340.800000000 -19302 414824.600000000 -7 414558.200000000 -2140 414435.890000000 -3897 414162.000000000 -5386 410059.520000000 -9386 409776.960000000 -713 409772.480000000 -5542 409678.500000000 -10178 408933.550000000 -18318 408350.160000000 -4135 408307.280000000 -9740 408248.880000000 -3223 407602.200000000 -9076 405293.900000000 -4486 403769.370000000 -16605 403305.900000000 -2586 403287.420000000 -13392 401370.790000000 -13122 401355.200000000 -6878 399595.460000000 -2711 399275.360000000 -16389 398905.000000000 -11428 397763.100000000 -12211 397286.640000000 -14211 397003.260000000 -1404 396559.680000000 -5106 395650.500000000 -14586 394990.500000000 -12073 394220.800000000 -18354 393073.920000000 -4182 391917.320000000 -17944 391714.400000000 -6275 389733.700000000 -12028 389573.040000000 -17283 388783.260000000 -3472 388739.220000000 -16320 388217.280000000 -90 387858.360000000 -8895 387658.530000000 -4374 386101.420000000 -8854 385400.730000000 -12528 384380.550000000 -16942 384250.640000000 -13365 384080.160000000 -13408 382965.680000000 -9898 379105.200000000 -755 378498.860000000 -6308 378400.200000000 -5267 376808.100000000 -7370 376441.280000000 -4866 375603.440000000 -8125 373807.140000000 -17013 371955.400000000 -17623 371460.600000000 -14116 371280.000000000 -12084 370901.370000000 -13579 370160.070000000 -5032 366346.700000000 -15858 365016.320000000 -5131 364325.780000000 -5403 363094.140000000 -5448 362910.800000000 -7481 361264.080000000 -52 358792.360000000 -15163 358417.280000000 -9555 358332.030000000 -253 357995.520000000 -13948 356288.850000000 -19347 356009.500000000 -3389 355636.580000000 -10367 354743.940000000 -1346 350637.430000000 -12603 349022.400000000 -408 348939.750000000 -7578 346501.120000000 -8370 344853.100000000 -5892 343870.800000000 -11846 340996.680000000 -3349 339738.540000000 -15627 339072.900000000 -15363 338539.050000000 -4822 337689.000000000 -12325 333570.380000000 -12087 333550.200000000 -4621 331292.600000000 -2615 330807.950000000 -2740 328308.600000000 -13839 326190.620000000 -6520 325524.130000000 -1617 324231.600000000 -14387 323321.010000000 -16629 322145.100000000 -15575 321494.320000000 -16964 320794.800000000 -18586 320402.100000000 -19403 317039.360000000 -15056 316535.250000000 -9369 315714.560000000 -14779 315580.650000000 -7954 315520.000000000 -8823 315432.360000000 -11932 315011.130000000 -8958 314998.080000000 -8586 313654.900000000 -19932 312586.560000000 -17368 311944.360000000 -4129 311286.400000000 -2727 308645.550000000 -18036 308176.070000000 -4257 306189.370000000 -4824 305727.800000000 -16618 303036.320000000 -15389 302119.300000000 -6796 301998.230000000 -13966 301049.760000000 -6130 297628.800000000 -5287 295215.360000000 -10252 294329.520000000 -6604 293585.940000000 -16622 292751.200000000 -5860 291985.000000000 -5070 291426.800000000 -14917 291102.520000000 -13432 290223.750000000 -11896 289787.190000000 -4372 288520.050000000 -12801 288085.600000000 -9736 287393.070000000 -13764 286449.120000000 -12349 285645.150000000 -5839 283663.520000000 -14007 282710.520000000 -2959 281456.000000000 -1225 280769.770000000 -5638 280292.420000000 -18755 280252.800000000 -17990 279389.350000000 -10659 278245.520000000 -11963 278048.940000000 -2076 275765.050000000 -14964 275425.000000000 -5840 275369.700000000 -15667 275267.200000000 -17691 275081.120000000 -6074 274454.460000000 -5167 274401.900000000 -4630 273177.900000000 -18050 272915.860000000 -15644 271246.560000000 -3309 271045.340000000 -2253 269714.040000000 -17083 268831.200000000 -17286 268754.010000000 -15138 268457.070000000 -5605 268349.200000000 -464 268085.700000000 -964 267485.460000000 -4714 267333.150000000 -12707 266687.500000000 -1632 266143.260000000 -7875 265152.360000000 -7139 265131.150000000 -14729 263650.450000000 -14356 261900.100000000 -18010 260543.580000000 -13406 260533.980000000 -14094 258814.570000000 -17677 258768.400000000 -11839 258542.830000000 -15795 257288.400000000 -7958 253638.000000000 -17548 253450.730000000 -18632 250603.040000000 -14883 249422.250000000 -13648 248378.200000000 -17765 248336.060000000 -3675 246654.920000000 -11697 245964.420000000 -3205 245462.250000000 -6963 244471.500000000 -2995 243620.880000000 -4597 243553.800000000 -10704 243357.480000000 -13243 243087.860000000 -4622 242676.000000000 -14171 242390.330000000 -6112 241420.320000000 -1628 239833.440000000 -1960 239649.930000000 -6306 238378.140000000 -12770 238264.670000000 -7226 237094.090000000 -21 236941.200000000 -11755 236331.650000000 -3641 236093.940000000 -7969 233819.200000000 -14430 233365.350000000 -8700 232934.000000000 -9793 231812.880000000 -3526 230570.300000000 -1928 230522.710000000 -11603 229962.030000000 -14794 229545.840000000 -1060 229364.520000000 -7557 228913.920000000 -10691 228843.420000000 -12336 228819.280000000 -17082 227969.070000000 -4755 227467.500000000 -14342 226880.720000000 -16876 226782.320000000 -17424 226333.800000000 -17355 225495.860000000 -6330 225197.000000000 -10444 224726.040000000 -6707 224680.560000000 -9713 224329.500000000 -7620 224175.080000000 -2867 220983.480000000 -12657 219172.400000000 -15339 218512.900000000 -5257 218373.690000000 -8383 215793.840000000 -10231 215061.300000000 -15086 214998.140000000 -11134 214758.900000000 -13586 214473.870000000 -19110 213151.470000000 -2814 211548.960000000 -8889 211100.670000000 -17326 210727.240000000 -9118 210289.000000000 -2743 210220.680000000 -6181 208024.750000000 -8190 207525.780000000 -3673 204191.520000000 -12729 204122.250000000 -1609 200863.960000000 -11368 200296.500000000 -10532 199656.060000000 -19725 198464.000000000 -10783 197926.560000000 -8779 197640.390000000 -7900 197577.620000000 -11364 194362.560000000 -10393 194167.200000000 -19076 193925.220000000 -11066 193589.720000000 -13593 193248.000000000 -133 193184.720000000 -10755 193116.400000000 -4419 192652.970000000 -8016 192357.480000000 -7815 189704.460000000 -8920 189388.300000000 -15869 188839.860000000 -18168 188835.840000000 -17088 188770.680000000 -12467 188733.800000000 -7192 188476.400000000 -8527 187128.120000000 -13644 186805.800000000 -639 186533.280000000 -7860 186430.080000000 -907 186159.890000000 -10067 185022.630000000 -13231 184250.520000000 -889 181904.310000000 -19115 181524.240000000 -2179 180659.850000000 -1242 179182.380000000 -4648 176590.700000000 -14368 176065.000000000 -19520 174676.260000000 -15740 173575.420000000 -9693 172042.270000000 -10157 170047.500000000 -15368 169315.120000000 -1349 168440.810000000 -9998 168255.120000000 -11149 166590.270000000 -9383 165160.920000000 -17986 162746.850000000 -6110 162588.240000000 -10869 162063.250000000 -12074 161659.140000000 -11389 161438.370000000 -5408 161423.880000000 -15586 161401.400000000 -9625 161061.930000000 -2791 161036.200000000 -7327 160098.200000000 -14139 159801.920000000 -19372 158688.450000000 -4450 158315.400000000 -4884 157226.880000000 -4133 157004.550000000 -3803 156503.640000000 -936 154045.800000000 -10523 153396.990000000 -4476 152675.000000000 -11125 152087.040000000 -10034 150861.420000000 -13386 149003.320000000 -18812 148899.280000000 -8181 147472.170000000 -16210 147247.300000000 -6562 146836.830000000 -436 146563.280000000 -1032 146408.400000000 -3032 144144.840000000 -18532 143103.200000000 -8758 142770.870000000 -8442 142071.600000000 -9231 142059.480000000 -13062 141412.120000000 -10600 141334.160000000 -19099 139067.100000000 -3055 138131.330000000 -3420 138030.240000000 -13349 137588.190000000 -7296 136659.200000000 -9661 136565.950000000 -8597 136552.650000000 -3587 136393.380000000 -3325 136229.730000000 -14332 134995.480000000 -14644 131902.890000000 -7872 131897.810000000 -19938 131830.260000000 -3613 130052.520000000 -19576 130044.380000000 -10473 129795.120000000 -3717 128415.370000000 -17101 127762.300000000 -860 127745.620000000 -12062 127627.620000000 -7068 127214.370000000 -12958 126118.240000000 -7018 126117.000000000 -10077 125380.400000000 -5389 125161.160000000 -18092 124733.940000000 -11077 124640.000000000 -19790 123414.000000000 -19257 122563.200000000 -1707 122400.000000000 -13064 120777.600000000 -8551 119975.590000000 -14852 118954.440000000 -13918 118924.200000000 -11954 118118.440000000 -9831 116576.750000000 -15433 116133.110000000 -16666 115973.470000000 -1636 114972.840000000 -2396 114097.200000000 -13353 113947.960000000 -1368 111906.630000000 -19350 111835.020000000 -6483 111282.850000000 -6351 111240.780000000 -9920 111216.050000000 -2830 110296.680000000 -14103 110276.640000000 -12830 108360.720000000 -18315 107893.980000000 -8040 107439.920000000 -2628 107236.800000000 -12949 107146.500000000 -10782 106793.020000000 -19078 106483.410000000 -19457 105913.350000000 -19787 104845.520000000 -4017 104569.300000000 -15067 104369.020000000 -586 103845.500000000 -9622 103326.300000000 -7281 101198.400000000 -6745 99053.850000000 -13051 98731.680000000 -6558 98153.280000000 -1729 97155.240000000 -9753 97004.840000000 -7845 95723.810000000 -5500 95642.560000000 -6360 95203.360000000 -17553 95037.620000000 -7129 94479.400000000 -3354 93871.800000000 -16562 93663.620000000 -5964 93171.920000000 -8345 92505.600000000 -11635 92318.940000000 -12690 91550.080000000 -10491 91345.330000000 -18084 91087.200000000 -5422 90654.090000000 -3081 90370.980000000 -3360 90031.950000000 -17275 89690.960000000 -8349 89400.090000000 -10143 89354.980000000 -4084 88330.880000000 -15330 87562.980000000 -1389 87438.910000000 -1476 86896.040000000 -9972 86497.230000000 -5198 86195.200000000 -14052 85855.680000000 -6132 85585.070000000 -7083 85162.520000000 -18598 85085.040000000 -681 84984.310000000 -13532 83267.600000000 -9434 82294.750000000 -16616 81461.640000000 -17588 81373.200000000 -6787 80627.240000000 -11867 78462.650000000 -14703 78175.500000000 -9994 78053.470000000 -2049 78016.400000000 -7410 77985.700000000 -16766 76765.400000000 -11157 76738.060000000 -1299 76722.600000000 -19207 75180.300000000 -16915 75013.020000000 -16413 74790.560000000 -19693 74245.080000000 -11554 73640.570000000 -6958 72982.560000000 -12141 72783.720000000 -18847 72720.720000000 -10868 72465.890000000 -5851 72159.060000000 -6645 71811.180000000 -15327 71625.600000000 -9380 71440.000000000 -15002 70963.200000000 -12858 70668.780000000 -18832 69991.500000000 -16381 69607.000000000 -17731 66137.050000000 -14958 66044.200000000 -577 65929.700000000 -3386 65797.190000000 -17602 65745.900000000 -7775 65371.280000000 -12653 64919.680000000 -1382 64638.420000000 -17028 64470.010000000 -6856 63921.780000000 -11644 62988.750000000 -10751 62470.740000000 -12403 61784.180000000 -3138 61465.800000000 -11180 60953.100000000 -5157 60481.260000000 -15045 60216.480000000 -11823 59405.720000000 -9211 58583.860000000 -14479 58195.060000000 -19053 57562.730000000 -1365 57470.400000000 -1095 57406.080000000 -19486 56736.020000000 -1958 56616.560000000 -11373 55984.560000000 -2156 55692.000000000 -18507 55231.800000000 -17498 54981.680000000 -19514 54231.240000000 -64 53189.280000000 -16179 52644.500000000 -12648 52342.680000000 -15833 52164.750000000 -15388 51211.270000000 -17398 50793.280000000 -17253 50621.760000000 -16147 50376.480000000 -12478 49894.020000000 -314 48762.300000000 -8953 48242.730000000 -11338 47212.840000000 -4211 47028.880000000 -2821 46928.900000000 -7300 46427.740000000 -13799 46054.970000000 -7532 46027.500000000 -3981 45978.660000000 -1541 45576.000000000 -3537 44017.400000000 -15821 43800.000000000 -18389 43748.430000000 -14312 43276.300000000 -2425 43175.840000000 -17169 42888.560000000 -5253 42329.150000000 -4854 42251.670000000 -19626 41949.410000000 -17058 41896.500000000 -5701 41342.400000000 -372 41157.760000000 -14271 41015.900000000 -16574 40958.500000000 -4671 40184.100000000 -10096 39102.250000000 -16647 38931.420000000 -10569 38663.100000000 -14535 38621.260000000 -14725 38535.240000000 -9606 38215.680000000 -19225 37457.300000000 -17327 37077.120000000 -5199 36498.240000000 -186 36468.750000000 -16854 35525.730000000 -15992 35492.000000000 -19807 34632.120000000 -11076 34028.880000000 -15641 33618.620000000 -11448 33411.520000000 -481 33101.800000000 -2984 33087.620000000 -13330 33085.800000000 -7257 32965.120000000 -10399 31781.750000000 -12811 31659.000000000 -19591 31213.820000000 -1723 30638.380000000 -16112 30103.150000000 -13994 29299.420000000 -2001 28345.000000000 -15622 28242.000000000 -3122 27982.500000000 -19789 27788.320000000 -10128 26191.320000000 -17206 25627.500000000 -8328 25540.560000000 -13950 25454.570000000 -6553 25389.700000000 -4957 25045.760000000 -19384 25028.840000000 -461 24725.220000000 -14821 24639.150000000 -16932 24418.470000000 -1428 23872.420000000 -15854 23789.280000000 -18081 23291.260000000 -10921 23263.240000000 -4073 22698.900000000 -2776 22492.190000000 -19425 21753.480000000 -14611 21369.150000000 -17542 20932.380000000 -1238 20893.180000000 -386 20884.240000000 -13560 20879.840000000 -19147 20462.270000000 +12098 16227681.21 +5134 15709338.52 +13334 15023662.41 +17052 14351644.20 +3452 14070870.14 +12552 13332469.18 +1084 13170428.29 +5797 13038622.72 +12633 12892561.61 +403 12856217.34 +1833 12024581.72 +2084 11502875.36 +17349 11354213.05 +18427 11282385.24 +2860 11262529.95 +17852 10934711.93 +9871 10889253.68 +12231 10841131.39 +6366 10759786.81 +12146 10257362.66 +5043 10226395.88 +12969 10125777.93 +1504 10004397.08 +14327 9981697.08 +134 9965150.66 +6860 9805871.26 +10624 9776138.40 +15819 9775705.31 +3293 9674928.12 +19865 9653766.83 +8870 9648981.87 +15778 9636332.82 +12360 9635023.92 +14389 9475588.34 +3257 9451029.24 +9476 9435207.28 +19629 9391236.40 +7179 9386222.25 +15723 9383900.80 +4054 9313810.02 +2380 9307751.22 +19084 9302916.80 +4703 9280804.80 +18791 9267017.97 +19994 9235972.92 +9149 9121803.90 +15118 9120819.50 +6116 9079369.20 +7052 9077468.92 +14147 9069193.78 +7305 9035228.53 +9130 9024379.25 +16698 8991337.95 +1553 8977226.10 +16777 8961355.62 +1402 8953779.12 +18963 8934063.40 +8358 8930611.48 +17547 8860117.00 +5128 8844222.75 +17063 8840649.60 +15490 8833581.40 +14761 8817240.56 +19601 8791341.02 +16160 8740262.76 +13597 8702669.82 +13653 8693170.16 +16383 8691505.92 +325 8667741.28 +8879 8667584.38 +10564 8667098.22 +17429 8661827.90 +17403 8643350.30 +18294 8616583.43 +4181 8592684.66 +13008 8567480.64 +13211 8537000.01 +1884 8532644.34 +11101 8530945.32 +11562 8528028.57 +15878 8523591.84 +834 8522135.27 +2423 8517902.85 +15383 8513433.11 +18119 8507611.80 +7389 8506099.20 +5016 8489784.15 +17473 8444766.24 +6669 8428618.46 +384 8418472.27 +12052 8411519.28 +17562 8409022.83 +8128 8379149.47 +13813 8374830.84 +12800 8318626.78 +10887 8315019.36 +1644 8285453.08 +16638 8274568.00 +1394 8255140.60 +7219 8254985.30 +13358 8253829.80 +5562 8252365.16 +14861 8242296.15 +15416 8196621.53 +1963 8192206.61 +2841 8148678.45 +6635 8122918.28 +3609 8099812.05 +6372 8093695.06 +5231 8091704.18 +8755 8085017.03 +4071 8083755.18 +4977 8058501.72 +11873 8057540.30 +12127 8051142.94 +2401 8049525.94 +15964 8037547.75 +10129 8030855.04 +7605 8028622.42 +9327 8022291.30 +11814 7983589.59 +4646 7981660.60 +6032 7981511.28 +1076 7977074.75 +4043 7971929.90 +8932 7967222.19 +13705 7953344.52 +16855 7923819.00 +3234 7920022.00 +17752 7901362.77 +2097 7892993.17 +18599 7890774.40 +19596 7874564.59 +11392 7861172.48 +18121 7857581.60 +17452 7838465.36 +6076 7821316.80 +15134 7804760.25 +8641 7802917.15 +2134 7800185.43 +16352 7797112.00 +19312 7775952.24 +2895 7759327.23 +12860 7758135.21 +153 7755681.28 +15089 7735438.26 +14797 7725353.16 +15946 7722773.88 +10919 7722425.36 +9867 7721597.78 +11881 7713136.42 +16552 7708518.15 +6925 7703999.68 +12147 7703826.98 +8923 7702690.28 +16116 7697970.84 +10661 7673830.20 +17094 7665368.16 +18648 7650862.02 +12172 7641326.40 +15123 7633032.50 +4993 7600570.80 +17162 7592062.56 +13506 7580809.83 +3436 7575616.33 +6271 7559793.93 +13314 7555156.63 +17242 7550949.50 +2753 7549574.06 +7391 7543159.68 +7418 7541449.65 +116 7520874.24 +12436 7520234.54 +1181 7494798.28 +12963 7491248.10 +213 7479470.28 +7114 7477681.20 +18521 7476478.30 +8973 7458603.67 +4202 7454095.74 +12009 7442105.40 +10609 7429346.40 +5622 7424142.66 +5143 7422760.28 +898 7414133.80 +12257 7408190.63 +6740 7400350.35 +1146 7394394.48 +5485 7378181.94 +8437 7376353.59 +6531 7362366.78 +16463 7362106.80 +10412 7359552.42 +12857 7340801.58 +12354 7332343.20 +7616 7320032.37 +3426 7312340.41 +8622 7307266.33 +6818 7304782.10 +3705 7299380.10 +12733 7298398.17 +1574 7293446.04 +10554 7289933.72 +9363 7284647.70 +4611 7282115.00 +7857 7266948.92 +9616 7265005.30 +15860 7254466.66 +15554 7247592.51 +3422 7247586.45 +9134 7236404.67 +17408 7220083.60 +15853 7219984.90 +9266 7218517.11 +1881 7208346.45 +10148 7205335.83 +8860 7202401.44 +8373 7189039.50 +10859 7188990.69 +12670 7188177.22 +2553 7180006.02 +19346 7176797.25 +1681 7160169.84 +15225 7158861.32 +1052 7158586.00 +77 7155531.10 +7231 7155250.38 +12622 7137408.42 +9814 7105363.14 +8695 7103187.00 +13174 7099182.53 +6179 7095134.05 +6451 7082495.36 +19860 7073206.83 +9307 7059973.68 +7819 7055963.04 +10556 7053491.07 +9366 7048690.74 +12124 7040021.31 +11476 7037906.76 +19245 7034045.24 +7562 7030275.79 +12290 7020372.06 +18118 7003396.80 +1253 7001569.62 +8662 6999834.27 +1779 6997385.73 +15386 6996871.79 +147 6989079.88 +9562 6983076.90 +3958 6969833.22 +7211 6966606.67 +12842 6923277.66 +4368 6918783.95 +11131 6918656.78 +4628 6894893.82 +5879 6881367.14 +16586 6865931.80 +32 6852925.59 +12119 6842773.70 +1371 6831137.52 +6136 6827917.01 +13857 6824240.60 +1074 6821747.88 +1863 6821522.19 +14597 6817385.66 +9271 6783068.88 +6389 6781075.68 +11703 6776538.36 +13701 6768880.56 +14880 6763788.24 +18428 6763670.54 +1006 6762065.94 +4927 6756765.21 +11659 6755246.68 +6815 6738928.35 +14367 6733857.20 +10703 6730936.46 +3150 6727920.40 +9963 6725919.35 +10438 6710153.62 +4745 6682153.67 +15297 6681711.28 +10848 6678666.25 +11749 6677895.73 +16739 6675549.12 +14915 6672248.66 +19841 6669191.20 +628 6666892.90 +1650 6657178.32 +7069 6648672.24 +7108 6646445.96 +8887 6641655.90 +18735 6636457.56 +3711 6632665.38 +2325 6630475.92 +6484 6622965.20 +2643 6617304.98 +7895 6615061.26 +12840 6604848.54 +4314 6600905.71 +19077 6591091.97 +17476 6576029.00 +7039 6559838.90 +8106 6558223.02 +2391 6557120.08 +7717 6547706.88 +12476 6546988.15 +9155 6540793.60 +360 6530297.41 +6383 6529336.02 +4830 6518998.92 +12600 6511549.46 +13740 6508057.92 +4678 6507847.98 +17815 6502284.76 +7329 6490811.95 +13884 6490063.10 +5147 6487069.00 +16548 6482024.50 +14144 6476413.40 +10181 6474984.88 +11031 6463308.02 +19958 6461506.38 +6043 6458177.64 +10060 6455476.89 +9144 6454042.05 +7043 6448019.98 +17346 6444307.52 +13963 6442014.48 +7111 6441947.07 +14140 6439955.54 +2327 6438977.28 +14812 6437152.54 +16755 6430895.36 +14840 6430549.14 +14134 6422079.26 +12655 6405496.79 +1518 6390148.22 +9888 6385033.02 +7387 6384005.18 +5393 6381083.04 +11057 6375974.22 +16818 6368828.80 +4576 6364925.71 +19644 6350000.33 +139 6336065.55 +11735 6334305.90 +10108 6332055.99 +15628 6329801.34 +4349 6314949.68 +7048 6313868.55 +17119 6298935.38 +18597 6291416.31 +2488 6286372.05 +2296 6275519.25 +4343 6272834.20 +9958 6267714.61 +2147 6267475.32 +9368 6254578.92 +13134 6252432.76 +10119 6251456.25 +2093 6249342.36 +2392 6237476.56 +17231 6233509.18 +3773 6226654.68 +9839 6214044.10 +19349 6213969.30 +7869 6212751.00 +8158 6210279.24 +13179 6205773.65 +2349 6205589.26 +9052 6200729.70 +1744 6189967.36 +597 6183103.47 +16721 6176606.60 +7498 6176277.25 +15157 6176222.50 +17524 6171107.36 +7922 6170906.07 +6615 6170738.42 +2706 6165991.65 +16432 6162740.68 +16473 6162427.96 +12825 6161595.60 +18813 6154678.55 +1030 6146500.28 +2571 6145772.43 +7707 6144754.71 +10327 6137612.00 +4710 6132346.56 +3649 6130602.63 +4893 6128461.24 +12844 6128191.24 +8794 6122690.28 +1157 6117749.22 +10895 6112017.68 +16166 6108250.98 +11920 6107122.56 +5621 6102123.56 +6141 6093826.56 +14076 6086671.08 +15884 6080485.59 +4814 6080337.96 +5814 6079842.96 +1134 6078685.20 +19048 6071813.28 +303 6070601.35 +15211 6065678.06 +1020 6054168.00 +11292 6052522.83 +7134 6049865.25 +14119 6049619.55 +2389 6042429.12 +5926 6034269.52 +8553 6030922.95 +18814 6023255.17 +12702 6023190.99 +2644 6020931.33 +19628 6010704.00 +18112 6008707.52 +13860 6008170.29 +1456 6005092.14 +1633 6002068.85 +2301 6000302.29 +10740 5999809.05 +2630 5997008.43 +8818 5992296.90 +10043 5990594.02 +653 5987942.83 +6829 5985990.66 +15179 5977727.52 +9663 5973523.92 +5863 5973328.92 +3628 5966340.09 +7618 5960155.86 +2588 5952648.56 +4865 5949383.40 +4233 5944699.60 +13390 5944104.69 +16321 5942714.70 +9653 5941308.50 +18884 5925548.24 +4394 5920927.14 +19774 5916723.12 +1257 5914052.36 +2963 5911917.77 +17157 5899573.02 +383 5884693.36 +11709 5884134.48 +18135 5871431.61 +13169 5869736.88 +2932 5868995.80 +2888 5863229.30 +6820 5853106.80 +18622 5850951.60 +9066 5846052.72 +19840 5832251.40 +6325 5827298.43 +14092 5823711.72 +11532 5823277.12 +18043 5815248.70 +3932 5809134.60 +10364 5808371.40 +1179 5808303.24 +11441 5799378.33 +15767 5798263.26 +14063 5797204.00 +11383 5793893.12 +10065 5781764.45 +17785 5766838.76 +18349 5761765.77 +14824 5760339.78 +14699 5759490.18 +11628 5755137.20 +4403 5752376.64 +13090 5751949.25 +15332 5744606.28 +17573 5744384.15 +12895 5741007.02 +13577 5739209.94 +16594 5732523.64 +8497 5727509.84 +2872 5724068.72 +16181 5721788.16 +6639 5712041.22 +13095 5708798.25 +4535 5693341.20 +10956 5692585.50 +19755 5686913.37 +12995 5682200.41 +13157 5681454.94 +1887 5681341.24 +18472 5680356.00 +19814 5679360.00 +18583 5669360.52 +3894 5664385.49 +1740 5659552.04 +62 5659470.16 +16532 5653779.46 +995 5648973.45 +7486 5646851.87 +19007 5642710.93 +13060 5642024.64 +12371 5635710.45 +2280 5634077.40 +3115 5631352.32 +11107 5631252.46 +5873 5629125.75 +14743 5628732.80 +2224 5624746.62 +2653 5623859.99 +17622 5623057.44 +14700 5615344.00 +14740 5613929.39 +6403 5611924.51 +6896 5609169.88 +10384 5607337.52 +16433 5605707.03 +5898 5604937.76 +4789 5600488.36 +8132 5593107.50 +3838 5592313.20 +13631 5586424.80 +11233 5585247.01 +849 5583516.45 +14653 5581550.45 +14788 5580433.00 +18181 5578562.88 +19815 5577102.62 +5584 5576692.20 +5385 5576420.19 +13780 5569028.61 +9342 5566783.71 +19056 5566524.12 +8189 5565694.74 +13808 5560721.96 +10635 5560058.55 +8304 5550784.41 +14257 5549164.06 +8999 5542100.10 +19134 5539312.56 +8360 5538031.10 +6397 5536651.92 +2597 5525317.76 +8631 5515909.38 +16729 5512663.65 +11861 5511785.93 +16853 5511689.91 +6341 5502790.08 +1312 5496649.40 +5566 5495885.87 +12519 5490649.97 +19032 5488105.02 +8231 5479312.50 +3026 5466732.32 +6388 5466168.80 +15349 5464571.52 +18985 5463897.13 +19848 5454266.40 +17378 5453284.74 +4000 5448690.39 +3710 5445822.53 +13181 5439774.06 +6420 5438325.32 +3644 5437772.14 +1117 5437024.97 +13027 5436968.46 +12884 5432632.34 +12781 5429161.08 +10084 5428231.62 +5640 5423318.58 +16208 5422901.40 +753 5416899.74 +4120 5413822.46 +12348 5412061.71 +1586 5411198.55 +2503 5411125.80 +1381 5397790.10 +19673 5397746.77 +19389 5394426.40 +15532 5386361.38 +3562 5380335.08 +19114 5375133.00 +3204 5372215.46 +6653 5365178.00 +3553 5363690.29 +12687 5361682.12 +3279 5357505.65 +9840 5350112.18 +8258 5347064.67 +11387 5345284.00 +15210 5341117.92 +15092 5340896.80 +6052 5339762.18 +14043 5339587.68 +6284 5336153.00 +6297 5332357.74 +16254 5326607.83 +18436 5326198.50 +14436 5325517.14 +10368 5319983.08 +6349 5317943.40 +19984 5317620.00 +19080 5310678.60 +1403 5306771.32 +5995 5305056.12 +13214 5299420.45 +19293 5297054.40 +7130 5289761.48 +9809 5286784.80 +9273 5277924.77 +16786 5260035.00 +11032 5256354.35 +17719 5246322.20 +3400 5244203.68 +8278 5243718.48 +7873 5241168.80 +9357 5239695.77 +2239 5235767.83 +18695 5223960.35 +19271 5216498.29 +4563 5214673.70 +1896 5210699.79 +15755 5210481.24 +17628 5209882.37 +5747 5206892.24 +18057 5204907.40 +5075 5204412.81 +5325 5190162.75 +17675 5183135.36 +18818 5181295.30 +11064 5180667.57 +12578 5177222.33 +3725 5169183.75 +11112 5161546.00 +4749 5161499.44 +7632 5159180.50 +13119 5151665.51 +5653 5140324.64 +16134 5137482.51 +5671 5136815.55 +18348 5132981.07 +562 5132487.25 +2562 5126763.83 +10304 5125180.00 +7622 5120719.98 +12755 5118213.92 +1814 5100705.58 +11269 5096671.33 +2964 5095253.72 +1616 5091834.00 +9294 5090753.53 +16793 5085330.54 +4999 5081651.75 +17209 5079029.28 +7151 5078937.60 +15522 5072469.60 +3056 5072329.55 +7612 5068322.87 +18453 5063892.92 +18324 5058901.22 +12266 5058186.75 +19394 5056235.73 +1713 5054968.05 +15681 5051569.63 +8274 5043328.00 +18160 5043074.83 +18253 5041572.00 +11840 5040590.04 +1532 5033171.00 +584 5031602.64 +12382 5028901.00 +14814 5022200.07 +19058 5019001.92 +4487 5016640.86 +8482 5015444.25 +18476 5011136.36 +12335 5003581.40 +4455 4997933.31 +14355 4992822.92 +15253 4992642.20 +14069 4983244.20 +17843 4977294.37 +9389 4975885.83 +14435 4971442.19 +13254 4959481.45 +9773 4955887.80 +7615 4952421.54 +6476 4947250.05 +9253 4945159.70 +14932 4934395.48 +13253 4932867.45 +19322 4931525.78 +16945 4931440.61 +731 4930191.93 +6540 4925114.51 +5148 4923048.00 +1934 4921196.90 +15402 4920840.72 +17914 4919607.04 +5416 4916041.92 +16734 4914205.27 +14967 4900262.08 +8706 4894595.58 +136 4891960.92 +19494 4886028.30 +8737 4880640.44 +7653 4879423.64 +4149 4875782.40 +7890 4872424.40 +11142 4871415.42 +10386 4863623.50 +8603 4861814.61 +2680 4861223.74 +4891 4858103.88 +19236 4855097.69 +14251 4854739.86 +18895 4853365.10 +17134 4852041.04 +4932 4843701.45 +10033 4841647.72 +1383 4839144.31 +18721 4837983.36 +8618 4833125.42 +17386 4831545.84 +3790 4830282.36 +1043 4825921.31 +12434 4822597.78 +18385 4819643.40 +6046 4817460.06 +5821 4814423.45 +10836 4814303.24 +6848 4813954.08 +6880 4804600.35 +11249 4800116.82 +11970 4799739.66 +14253 4796521.29 +7782 4793227.13 +75 4790042.88 +7076 4789347.34 +9566 4782531.80 +2137 4767931.74 +2336 4763870.79 +15362 4759043.38 +3284 4755048.76 +12964 4753627.48 +1781 4752835.20 +5454 4748342.98 +12597 4742077.84 +19120 4733459.96 +14884 4731499.44 +825 4730720.28 +14683 4730482.32 +5361 4726113.00 +12179 4725018.10 +1461 4710954.69 +9890 4709658.40 +13369 4705085.39 +11612 4701627.99 +3096 4699414.40 +10383 4697866.47 +11293 4697336.06 +3383 4695825.20 +6231 4694381.72 +7396 4691319.06 +17827 4688797.44 +15856 4683368.21 +8253 4678560.86 +12327 4677984.74 +4395 4676829.82 +4232 4676646.40 +14260 4670522.80 +15288 4669273.99 +17526 4668545.64 +9884 4662693.84 +2118 4660352.78 +4524 4653956.60 +19090 4650872.94 +3928 4649359.44 +14325 4647762.17 +15476 4643469.04 +4179 4639931.76 +14408 4639631.28 +19424 4634817.44 +3334 4633102.50 +9477 4628073.51 +11803 4625396.80 +14805 4618452.18 +463 4616307.28 +16628 4607490.96 +3116 4604463.10 +19962 4602949.47 +12859 4602870.55 +12063 4600708.45 +5648 4592273.25 +8556 4590726.86 +15281 4589425.41 +9414 4587426.90 +13951 4586281.25 +19328 4582624.82 +15963 4579705.50 +10773 4573276.20 +14179 4568816.00 +1895 4563988.16 +6408 4561496.39 +5958 4554000.00 +3653 4548134.40 +11218 4546237.92 +19327 4543987.77 +9572 4535941.16 +14556 4531464.75 +2475 4529761.50 +9631 4529261.56 +1901 4528592.55 +86 4528475.38 +9586 4527146.22 +17361 4519098.87 +8112 4514949.45 +13468 4499728.20 +18239 4497633.64 +10215 4494553.60 +6211 4492264.96 +836 4490945.10 +895 4489141.50 +19542 4488393.75 +4322 4487884.23 +2116 4486944.65 +553 4486075.48 +2515 4485188.26 +16286 4481470.47 +12271 4478224.95 +16570 4465818.00 +7995 4457574.66 +18396 4457229.60 +16331 4455735.48 +18157 4452196.63 +5271 4452040.01 +11622 4451244.84 +4052 4446397.34 +2864 4446008.38 +490 4442892.30 +19837 4434172.39 +4114 4433657.85 +11436 4433070.15 +6085 4431306.57 +9735 4430445.60 +17834 4416286.33 +8157 4416116.65 +18840 4414925.32 +13553 4412261.70 +12562 4411183.04 +14025 4403442.16 +17964 4400360.09 +636 4399863.84 +8390 4389024.33 +231 4387397.30 +9699 4385891.02 +10622 4384005.32 +14364 4383236.90 +10580 4381533.23 +10124 4369800.96 +10451 4368867.50 +4673 4367113.44 +11351 4362616.50 +4770 4362397.32 +12932 4362042.60 +10603 4357216.50 +19733 4348931.75 +4222 4348871.91 +17319 4347687.69 +3375 4346529.48 +14995 4338295.65 +7675 4337499.60 +15043 4333921.20 +4835 4332648.00 +4408 4332588.90 +5559 4330577.09 +7376 4328936.54 +18061 4328793.98 +2749 4328671.53 +6628 4328501.88 +5888 4323049.72 +18872 4322595.62 +5476 4319642.58 +1755 4318935.63 +10623 4315822.56 +18775 4314677.64 +3570 4312697.87 +11147 4310740.57 +6071 4307612.40 +10807 4306006.00 +9550 4299478.56 +657 4296794.19 +19669 4294640.90 +8532 4290651.60 +13469 4281715.62 +8809 4280778.80 +11301 4276847.95 +6147 4266879.92 +2612 4265962.35 +15699 4256118.72 +12300 4254409.11 +3494 4250810.60 +11040 4250030.20 +6190 4244046.80 +17616 4239937.50 +7271 4234407.00 +14048 4226977.44 +4456 4224684.98 +10012 4223841.21 +11175 4223704.14 +18675 4215406.86 +10792 4214898.57 +10806 4209678.32 +18749 4204787.00 +17410 4198025.28 +8032 4195430.00 +11094 4192304.94 +17582 4187341.44 +12246 4183230.95 +6640 4182968.80 +7346 4174707.60 +12747 4169865.81 +3869 4164957.44 +13106 4161902.08 +10547 4159541.36 +15289 4156205.76 +1679 4156156.64 +1126 4155593.08 +19106 4147439.52 +9705 4144024.20 +15324 4142518.56 +16544 4140375.72 +8812 4139322.81 +10772 4134101.64 +2800 4127150.08 +15549 4124704.64 +3607 4118697.57 +1980 4117633.72 +214 4113117.36 +19217 4104217.60 +2460 4098577.46 +19156 4093864.46 +18359 4092727.29 +12865 4092526.84 +14616 4092434.54 +908 4088856.20 +11791 4083804.97 +4157 4078345.60 +3857 4070872.87 +15114 4056112.50 +395 4052997.76 +17456 4051457.28 +10562 4050894.19 +10884 4050330.76 +12177 4049842.68 +15595 4040577.56 +15916 4036044.50 +7084 4035102.72 +4424 4034761.56 +10874 4031015.85 +4740 4030403.76 +16585 4030010.26 +18824 4028984.10 +14875 4028452.08 +13855 4024828.34 +10932 4024002.40 +9084 4021362.45 +14352 4018089.74 +18086 4015180.68 +9514 4013666.67 +15787 4013154.56 +714 4010249.44 +8811 4009588.90 +14386 4007210.88 +616 4004057.26 +7460 4003412.48 +866 4003182.54 +782 4001299.94 +8562 3999441.62 +1366 3994060.86 +2879 3993056.55 +16679 3992434.99 +17306 3990723.30 +13140 3982817.39 +17942 3980857.04 +6572 3977676.28 +3578 3977523.94 +15802 3969946.90 +336 3967938.38 +9807 3964469.60 +12104 3964273.40 +4271 3962359.28 +6702 3961657.44 +19763 3955582.75 +369 3953702.88 +4089 3953455.68 +2593 3946153.80 +590 3943841.16 +8325 3942118.75 +158 3941881.65 +12054 3938362.69 +18330 3938303.88 +5354 3936239.58 +8150 3925793.46 +8344 3921293.60 +6069 3921130.55 +4032 3920008.59 +17939 3917750.27 +7014 3914471.20 +2840 3913131.58 +1868 3912987.54 +10975 3911920.48 +5374 3910802.74 +11128 3908156.46 +18449 3907589.40 +11740 3907459.84 +2356 3907189.00 +5721 3901585.97 +4231 3900779.05 +4352 3899933.44 +432 3899836.44 +15321 3899516.58 +10296 3897015.14 +5647 3895088.16 +7386 3891916.51 +507 3891487.68 +3995 3887387.07 +4278 3882294.02 +18407 3880267.86 +6127 3879166.71 +145 3875277.24 +19269 3874685.76 +18257 3874454.89 +9068 3869767.74 +576 3860007.79 +4860 3852862.02 +18793 3849838.16 +15988 3847257.05 +6891 3846386.75 +3231 3846344.30 +15237 3845421.00 +9035 3844166.85 +7597 3838643.35 +16349 3837121.65 +2497 3827850.20 +3616 3827390.95 +11566 3826122.47 +18403 3822033.02 +2972 3821903.55 +812 3821523.72 +2043 3820561.36 +505 3818922.03 +8257 3815071.92 +6084 3814194.95 +11253 3813917.24 +366 3812257.88 +13632 3811601.32 +14298 3801412.42 +7092 3798729.48 +2058 3796109.04 +14820 3791195.86 +7157 3788690.82 +17211 3786030.17 +16644 3786019.25 +15693 3783662.19 +2627 3782394.60 +11231 3782077.60 +12696 3781761.66 +8705 3778077.00 +16052 3771577.04 +99 3760269.31 +2082 3757517.50 +872 3750005.34 +7126 3749138.92 +10302 3744475.25 +17122 3741012.98 +10080 3740107.10 +16021 3739611.20 +3074 3739224.96 +3142 3738811.02 +13213 3735116.25 +13442 3733132.14 +11542 3731000.12 +13732 3730444.90 +2608 3729372.40 +5 3725511.50 +19157 3723844.72 +18231 3721707.99 +8179 3714155.04 +12740 3708646.91 +11597 3706528.59 +13968 3702376.08 +6436 3687346.44 +9181 3687134.08 +564 3680200.80 +13464 3678406.20 +14084 3673790.38 +2755 3670593.69 +14284 3668640.80 +12178 3653392.48 +15730 3650258.30 +5560 3649569.59 +8594 3647140.56 +7032 3646439.54 +16846 3644843.10 +1530 3642838.08 +3978 3639712.05 +2897 3639442.32 +16625 3636527.54 +12029 3636339.72 +16830 3633448.57 +9597 3632662.11 +5533 3630338.67 +5181 3625965.93 +8131 3625738.62 +8560 3620761.26 +11860 3618746.25 +12008 3614604.40 +10737 3611990.64 +18208 3611596.10 +5119 3611038.20 +11958 3601654.65 +15124 3598278.20 +14058 3597490.02 +12270 3593912.10 +17793 3593318.95 +9385 3587327.84 +12814 3587083.84 +5304 3586230.61 +3631 3582841.65 +610 3581917.30 +19317 3580412.43 +128 3567004.56 +11616 3566154.80 +10176 3565392.15 +7349 3564110.64 +1712 3560408.43 +18860 3559340.60 +17617 3557516.00 +6443 3556296.96 +15408 3554814.56 +16350 3554388.63 +17436 3554105.13 +5740 3551324.68 +12181 3550218.54 +16895 3550119.30 +19995 3548839.70 +4968 3548306.87 +2257 3546692.29 +1825 3543198.78 +18989 3539038.08 +18727 3536081.40 +16165 3533789.84 +3249 3533709.87 +11731 3532875.00 +13032 3532415.79 +9377 3531582.08 +5883 3531479.00 +1211 3528833.40 +12065 3526948.10 +10866 3526146.66 +2073 3520131.30 +2378 3512186.20 +16860 3509693.07 +389 3507814.64 +15604 3505653.27 +11257 3502831.80 +1327 3502022.60 +16602 3501074.88 +1493 3498808.95 +8224 3498179.52 +622 3497158.36 +3072 3495958.72 +1478 3494880.48 +3125 3494169.90 +2052 3488438.08 +8476 3487191.28 +10735 3477740.76 +14860 3476235.84 +6586 3475745.10 +5130 3472024.50 +7181 3471306.30 +618 3467906.52 +15698 3464859.47 +17585 3462450.46 +2548 3456856.96 +2632 3456230.74 +2882 3453986.86 +12216 3452907.15 +4925 3452904.63 +9012 3442581.36 +6667 3430076.40 +17958 3424962.56 +6093 3424241.92 +10648 3417414.00 +1462 3413248.61 +2569 3412388.82 +18616 3409880.91 +7368 3408036.45 +3110 3407374.60 +10824 3406819.29 +11510 3404701.96 +4840 3397236.40 +4449 3396993.60 +1358 3396616.32 +3885 3395817.60 +13381 3391953.52 +1655 3383051.51 +282 3381785.42 +4928 3374270.48 +3199 3372488.80 +16086 3370710.65 +8612 3362922.50 +19597 3360764.00 +8867 3354400.11 +4098 3353574.28 +12617 3351499.05 +14365 3347296.00 +10443 3345493.10 +76 3342081.82 +11585 3341941.22 +4383 3338960.27 +13910 3335964.16 +8076 3332449.89 +16005 3332190.40 +2622 3329364.45 +12822 3321183.52 +17076 3320398.06 +5392 3320357.15 +18628 3319615.84 +13695 3318525.99 +10326 3318274.16 +9109 3317833.90 +1489 3317620.80 +3378 3315948.00 +7738 3312979.20 +1844 3312277.36 +19963 3307500.00 +2436 3306419.05 +886 3302180.70 +15475 3301693.50 +6327 3300680.78 +6050 3299460.20 +9876 3298410.05 +19586 3291131.25 +14349 3289862.52 +10993 3287980.57 +18784 3286752.12 +1800 3285466.24 +990 3284595.50 +3823 3281992.94 +15737 3279305.96 +19518 3276759.63 +9032 3272440.32 +7786 3271217.28 +8648 3271162.44 +5532 3270187.97 +15914 3268520.98 +16065 3265068.84 +11212 3264657.03 +13229 3262022.28 +15827 3260862.72 +1582 3260340.00 +3827 3260093.76 +3546 3259244.07 +15849 3258918.00 +14856 3258379.40 +2028 3255013.96 +6618 3254581.95 +17461 3252926.88 +13551 3241602.20 +19561 3239795.32 +2276 3236172.30 +14203 3234649.39 +7757 3231351.84 +122 3226213.88 +12954 3225943.00 +647 3224783.76 +12383 3223989.44 +3831 3223126.60 +16836 3222260.73 +4565 3221597.44 +19426 3218106.54 +17855 3217813.02 +5624 3207777.36 +8368 3203376.45 +9480 3200904.00 +11181 3199500.53 +8981 3197864.00 +16426 3195995.97 +1648 3195558.90 +14404 3192729.60 +17867 3188571.00 +18117 3183229.04 +14289 3182261.60 +53 3182256.00 +15546 3180180.04 +16245 3178277.46 +1597 3176247.48 +1653 3173456.64 +2845 3171619.61 +15906 3171187.54 +18304 3168571.50 +14068 3167367.60 +6837 3165012.48 +9446 3164446.52 +18889 3156140.96 +16587 3154210.20 +7705 3152977.38 +1120 3151591.17 +17665 3148848.00 +5311 3146721.86 +14157 3144707.32 +7996 3131351.04 +8663 3130526.32 +18271 3127800.96 +6446 3125685.96 +6972 3125007.06 +2572 3123186.83 +13536 3122527.54 +6196 3122172.48 +9338 3121262.40 +11992 3118647.55 +2580 3118284.37 +9098 3117494.10 +5118 3112661.96 +10184 3109293.40 +9932 3105818.24 +18545 3102273.32 +10963 3099314.50 +8405 3097121.12 +9037 3095195.00 +179 3091107.28 +1930 3090915.80 +17723 3090624.66 +4308 3089472.75 +8702 3080129.92 +18621 3079984.80 +4501 3079781.10 +3590 3079049.42 +18264 3078858.44 +15648 3078564.06 +5998 3073264.00 +16904 3072610.80 +3794 3071333.09 +3147 3068485.32 +17221 3068337.22 +4709 3067523.31 +18017 3066743.41 +15613 3063987.86 +16271 3057051.34 +13621 3054774.59 +12919 3054518.50 +12493 3050836.30 +15838 3050645.95 +3273 3048955.15 +8324 3046011.25 +13628 3045324.50 +5522 3044408.50 +2202 3043132.05 +19052 3042566.55 +5767 3041871.70 +17895 3036452.22 +12586 3036386.30 +12425 3035041.52 +13517 3034351.47 +2363 3033336.60 +15060 3032598.51 +6764 3032591.10 +340 3030522.00 +4723 3028910.25 +3566 3027858.61 +17796 3026838.96 +15384 3023792.64 +16336 3010813.56 +679 3010713.30 +7554 3010667.80 +14553 3009756.96 +8379 3009745.17 +15436 3007499.77 +12471 3003991.86 +18059 3003037.53 +8536 3000746.00 +19033 2999373.28 +18179 2996151.20 +10711 2996143.17 +17271 2994264.79 +13932 2989023.58 +3101 2987788.16 +14550 2977853.65 +3080 2977232.58 +14533 2976490.49 +14439 2975313.24 +9237 2973124.78 +1205 2971470.28 +12361 2963419.47 +429 2962631.88 +3970 2960418.45 +8403 2957698.45 +1098 2957514.00 +7932 2955046.14 +16266 2952298.38 +19386 2948854.48 +13147 2947037.91 +2720 2947011.08 +3840 2944219.35 +13482 2942474.88 +9436 2940396.21 +19779 2937105.96 +18032 2933224.38 +7743 2932733.77 +14620 2930766.89 +4606 2927832.59 +18076 2924134.83 +19276 2918176.20 +7483 2915918.95 +8575 2915132.64 +11052 2913140.88 +17251 2908345.80 +8788 2907935.93 +10960 2906511.14 +18782 2903643.78 +19988 2897461.53 +726 2896009.27 +19942 2894251.36 +10864 2892252.48 +17840 2891563.22 +18717 2888939.96 +12391 2886051.30 +18219 2885921.06 +15100 2883342.33 +2491 2880385.74 +12389 2879696.96 +3880 2877770.24 +18579 2874542.48 +13647 2873838.34 +15758 2873804.92 +12917 2873659.60 +18866 2873616.26 +13894 2872986.12 +15200 2872571.93 +9628 2872404.56 +8568 2871598.08 +8389 2870237.88 +5788 2867210.18 +19450 2863310.66 +9440 2863162.92 +16795 2860135.41 +19643 2858987.80 +1974 2856825.84 +14622 2852089.12 +6885 2851437.62 +12532 2848992.64 +1087 2847858.80 +5777 2846407.41 +5629 2846076.12 +6316 2840544.65 +12767 2840514.12 +12134 2840036.91 +14476 2839853.01 +803 2838388.16 +18964 2836942.44 +6020 2833459.20 +10401 2832688.74 +1323 2829964.50 +1151 2829662.44 +1458 2824034.43 +2271 2820756.53 +18740 2814140.80 +7348 2811730.95 +4281 2807190.52 +8043 2804706.24 +3843 2804217.96 +7813 2802350.88 +347 2802245.52 +745 2801725.10 +10388 2799170.58 +18100 2793358.50 +19043 2789013.80 +10644 2787797.01 +16170 2787402.80 +398 2782729.05 +9370 2780078.13 +14504 2780036.04 +1210 2778485.76 +13385 2777445.62 +3799 2775223.60 +11325 2769766.02 +3489 2769554.52 +17181 2769028.50 +6964 2766653.78 +7381 2764898.80 +6253 2764394.64 +5975 2760819.72 +11996 2760687.86 +7570 2758977.12 +4387 2757672.00 +9014 2755367.42 +9403 2748021.66 +11653 2739731.07 +17697 2739312.29 +958 2738032.00 +18816 2737140.00 +14104 2735008.64 +15966 2732250.20 +17912 2724160.95 +7089 2720170.04 +16032 2718976.18 +16891 2717293.32 +19579 2716909.86 +17470 2715048.84 +12408 2712556.52 +4763 2711800.90 +1138 2709709.81 +7363 2708414.40 +7877 2705439.45 +17532 2703698.68 +10512 2701235.92 +11957 2700133.22 +2455 2699593.88 +15119 2696860.80 +9868 2696801.52 +14172 2695307.48 +16120 2689337.82 +13958 2679025.28 +15169 2676686.04 +2648 2672232.00 +6164 2671317.32 +12701 2669216.40 +16382 2669034.54 +15588 2667212.10 +14830 2666758.15 +9119 2665812.24 +1622 2665206.50 +878 2664045.79 +13269 2662784.12 +619 2655417.63 +18386 2653795.02 +2501 2652260.40 +2310 2651631.09 +19420 2649395.61 +4895 2645152.27 +7553 2643682.07 +17814 2642781.44 +16097 2642500.00 +10995 2640811.16 +14895 2637733.72 +18546 2637026.71 +9875 2631358.80 +9591 2626899.54 +6001 2625893.76 +7739 2624573.28 +10431 2624379.54 +4544 2615313.75 +16387 2603195.76 +18375 2601407.83 +8395 2598728.44 +18853 2593356.36 +4900 2592813.15 +1302 2592197.76 +17032 2589806.40 +14292 2589749.56 +43 2587359.58 +5221 2587024.04 +397 2579751.46 +17890 2579674.24 +12157 2575510.48 +7340 2574645.83 +19368 2572618.95 +8848 2570819.15 +13789 2570243.26 +14596 2568234.24 +8408 2567434.41 +19726 2565750.42 +13964 2565579.12 +7740 2563027.50 +14768 2560392.60 +11734 2559062.22 +10294 2558257.97 +15032 2557926.22 +9127 2556379.80 +2181 2553175.00 +16653 2552229.68 +3866 2549994.79 +16814 2548710.76 +1866 2545838.40 +3512 2532626.80 +4145 2529786.15 +12120 2528298.72 +644 2528123.05 +15379 2525181.01 +6392 2524063.08 +2652 2521456.80 +3363 2519202.23 +19167 2517993.18 +16042 2516599.92 +2892 2511854.40 +5711 2509401.72 +14591 2506344.69 +6564 2506277.34 +1231 2505421.24 +5049 2502603.00 +14576 2501606.69 +10211 2500852.20 +293 2493168.48 +7371 2491134.65 +18154 2491047.20 +9494 2489825.52 +14836 2480432.40 +19471 2480403.75 +802 2478998.33 +12541 2477242.60 +15065 2473563.94 +15995 2472803.20 +9408 2471953.56 +9776 2470447.90 +17325 2468989.05 +3391 2468317.72 +16123 2467022.22 +18758 2463798.06 +407 2460304.47 +6840 2456170.78 +9995 2455155.36 +3877 2453696.65 +5817 2452493.13 +14122 2452226.22 +16699 2450273.98 +8921 2450116.48 +15103 2449861.20 +7637 2449628.72 +3076 2443927.38 +6648 2443248.95 +17116 2442263.72 +1645 2440838.40 +3181 2440017.60 +5966 2431558.08 +15882 2428947.30 +7529 2428381.28 +12836 2427897.33 +18052 2427637.76 +13616 2426638.50 +16615 2424775.08 +18147 2424412.68 +4586 2424123.90 +14403 2423141.96 +11606 2422794.31 +13526 2422212.80 +3677 2421404.46 +5553 2418506.21 +12109 2416514.17 +13118 2415931.80 +1563 2408855.40 +16591 2408045.39 +6411 2404918.53 +10272 2402834.48 +10597 2400247.68 +13700 2398035.86 +9548 2397147.90 +14963 2395781.09 +13325 2390637.58 +13864 2388067.88 +7450 2383447.71 +9275 2382868.40 +5829 2378037.92 +13437 2377806.54 +13594 2375046.30 +11442 2374591.08 +15619 2374052.38 +9063 2374035.84 +5990 2368686.50 +7811 2363829.26 +9525 2362974.53 +5597 2361031.84 +8963 2360774.00 +1709 2359839.29 +15814 2358656.64 +17613 2357519.04 +5022 2354550.45 +17740 2354242.83 +3388 2351042.26 +13773 2348739.12 +14467 2348665.04 +11544 2345324.45 +349 2344664.13 +10356 2340862.72 +18272 2338754.60 +4627 2337430.84 +327 2335298.46 +19846 2332224.73 +10814 2330319.60 +13102 2326122.75 +18867 2323972.00 +2824 2323315.08 +19117 2319911.10 +1906 2319757.60 +245 2319450.90 +17318 2317860.39 +3862 2316453.72 +8100 2313874.12 +2958 2312239.47 +10263 2308514.06 +13814 2304940.40 +9394 2303161.74 +18080 2299416.78 +1271 2289526.98 +3327 2278474.48 +8740 2278405.92 +8119 2276428.17 +3368 2274373.62 +7963 2272300.80 +2151 2270932.72 +16995 2270264.68 +9918 2269733.07 +503 2268535.25 +16692 2256484.50 +793 2254198.72 +16455 2252361.86 +6644 2249521.82 +17280 2249437.50 +6813 2248982.00 +4674 2246915.32 +16325 2244369.80 +182 2243290.00 +4626 2242474.35 +10860 2241291.60 +14034 2241220.80 +2476 2240855.20 +4253 2239985.64 +3211 2239871.02 +1290 2233313.00 +8479 2232189.04 +11895 2231607.00 +3487 2230171.62 +14870 2229915.37 +16328 2229483.96 +18585 2228215.50 +7638 2228208.08 +5436 2225672.28 +14594 2223005.07 +4532 2215711.02 +7586 2210562.51 +11870 2205182.82 +18487 2203653.60 +9179 2202720.52 +16500 2201185.31 +3679 2200592.70 +12803 2198295.00 +18056 2196741.90 +11396 2195645.64 +5087 2194120.72 +8067 2192048.64 +15357 2191646.58 +4491 2189713.50 +208 2189046.80 +10958 2188766.82 +9126 2188410.50 +15084 2184327.02 +18850 2183309.52 +3398 2180250.00 +16137 2177318.76 +211 2174808.96 +18422 2174381.00 +15840 2173510.40 +19553 2173079.77 +8221 2169992.16 +17000 2169611.16 +6755 2168505.15 +10817 2167710.68 +8327 2167650.60 +543 2167368.00 +4553 2163371.52 +15019 2162288.00 +334 2162178.48 +8516 2161479.04 +11349 2158941.88 +3902 2157027.86 +14731 2155302.24 +326 2153380.08 +11403 2151242.30 +11657 2150446.08 +9496 2149219.01 +8110 2149120.13 +5153 2148527.25 +884 2148324.98 +8637 2146185.10 +2364 2145790.72 +12386 2145001.47 +10133 2144903.96 +9895 2143324.80 +13755 2142539.40 +4327 2138501.40 +3369 2137408.76 +5815 2136985.00 +19357 2132657.28 +2675 2124158.72 +17869 2123991.72 +11702 2122132.99 +17257 2117850.64 +9952 2116686.32 +3881 2111457.15 +10951 2111185.58 +2128 2109702.30 +6699 2106578.40 +3155 2103636.64 +16649 2101956.20 +15257 2100297.75 +9978 2099566.56 +16810 2098301.44 +10653 2093388.70 +10476 2092766.48 +10883 2087495.28 +9704 2086967.61 +1119 2085182.84 +19139 2079788.34 +2144 2078391.14 +9135 2076377.80 +18548 2075584.32 +10545 2075230.35 +6220 2074341.72 +8616 2072887.65 +5230 2072161.74 +13916 2070504.72 +4299 2069922.96 +894 2069688.16 +17847 2063367.04 +18879 2061902.25 +13036 2061600.17 +10606 2060492.40 +9454 2060016.48 +118 2059808.86 +9601 2059715.76 +13769 2057668.08 +1987 2057289.27 +13863 2055368.00 +13562 2054754.24 +1840 2054183.92 +17995 2053221.90 +17389 2051128.20 +15168 2045987.49 +2139 2045365.40 +4024 2044243.10 +8964 2041648.85 +181 2040167.04 +7628 2039548.92 +3 2038846.09 +15553 2036958.91 +11355 2035405.60 +13006 2034991.20 +3091 2031393.51 +1281 2030628.48 +1408 2028621.66 +18211 2024538.67 +2287 2020754.32 +6228 2019198.82 +4362 2018495.25 +10873 2013280.32 +7383 2009581.92 +1386 2006544.26 +9820 2005815.76 +18134 2003409.73 +15727 2000654.50 +157 2000148.16 +19571 1999891.11 +17728 1997944.40 +5278 1996644.21 +17737 1994653.76 +10220 1989890.98 +1397 1984509.22 +6195 1983928.26 +4270 1983726.95 +16965 1983286.25 +1683 1980638.64 +13086 1978609.40 +7124 1974039.38 +5211 1973843.76 +6794 1973149.47 +257 1973035.44 +6995 1968281.55 +8447 1967292.70 +15873 1967257.89 +12862 1964014.13 +8295 1961467.08 +931 1958825.22 +6876 1957359.48 +1932 1954592.40 +1061 1952688.06 +18108 1951143.67 +5138 1950861.00 +12598 1950211.61 +10829 1943924.62 +11950 1941211.00 +12076 1939323.96 +2176 1938691.37 +6616 1937401.88 +5893 1934358.58 +976 1933066.80 +13173 1932557.52 +14947 1929229.98 +16857 1928814.80 +13403 1928702.88 +4819 1926969.68 +13127 1926929.83 +6871 1926787.68 +15465 1925145.09 +1131 1920005.50 +11845 1913576.40 +8364 1909122.20 +16588 1904272.37 +6759 1903906.29 +11586 1901895.65 +8145 1901787.66 +17333 1897297.20 +13290 1890633.75 +6499 1887621.00 +4881 1887535.92 +7147 1886710.20 +3883 1886567.78 +18911 1885597.12 +11336 1883573.60 +8653 1883275.76 +19476 1881492.48 +14799 1880543.40 +14491 1879219.92 +11815 1877434.34 +3173 1874302.10 +7161 1873023.45 +14631 1873015.30 +4247 1869912.96 +3568 1865824.40 +1500 1865451.03 +11833 1863665.23 +495 1860771.30 +6776 1855589.17 +11374 1855221.12 +5637 1853782.17 +3597 1852826.80 +981 1852083.60 +16076 1850349.69 +17597 1845420.95 +19609 1843185.48 +10997 1843072.02 +3403 1842975.00 +897 1842845.10 +16697 1840630.68 +17644 1840597.80 +6485 1838812.02 +5492 1836202.88 +12038 1835075.06 +9325 1832634.84 +10637 1832347.44 +11318 1830158.39 +4357 1828730.00 +18553 1826335.20 +12623 1825950.85 +961 1825869.60 +1677 1821816.90 +8211 1820432.52 +19719 1819333.55 +19663 1819074.35 +16296 1818353.77 +16527 1817834.42 +4964 1815400.02 +1769 1812929.20 +13126 1808799.96 +7854 1807608.06 +18380 1803641.22 +6584 1802346.98 +7665 1801765.35 +16553 1796146.78 +17761 1795095.72 +11179 1794890.30 +15171 1794148.72 +3018 1793183.88 +15741 1788612.00 +5331 1783901.35 +9860 1775071.26 +7984 1774302.75 +15354 1774270.77 +17884 1774212.44 +16257 1771869.71 +10696 1768645.20 +2104 1767902.64 +14465 1764946.40 +10089 1764692.32 +6719 1762699.54 +3648 1760594.42 +7241 1759913.59 +11122 1757430.04 +17019 1752560.65 +13877 1744271.10 +15325 1743826.26 +17860 1739870.44 +2236 1739795.80 +4436 1738760.32 +7701 1738670.40 +8147 1736855.16 +6676 1736341.44 +19505 1735413.43 +9885 1731366.26 +2112 1725934.08 +5330 1722196.98 +3561 1720377.96 +10104 1714419.16 +16362 1712457.38 +15573 1712365.44 +15006 1711381.35 +14629 1709942.05 +9612 1709528.38 +19910 1709211.15 +13145 1708907.46 +11494 1707973.68 +15895 1706999.45 +8239 1705479.10 +2403 1705331.10 +19436 1702706.00 +3476 1702335.80 +6828 1702292.08 +771 1701589.50 +8448 1700312.44 +3755 1699047.03 +13895 1698679.03 +9785 1698056.37 +6180 1695571.53 +532 1694356.15 +6741 1692552.42 +19964 1692367.64 +3747 1691244.60 +3253 1690719.42 +16119 1688339.25 +7113 1681911.00 +12368 1681219.80 +16378 1679705.60 +1393 1675545.35 +11119 1675453.44 +4469 1674023.49 +6955 1672618.90 +11579 1672345.32 +19898 1671781.70 +15351 1659204.30 +6133 1658215.46 +9110 1658054.68 +2979 1656016.74 +18764 1653708.48 +8995 1653627.58 +13096 1651408.67 +15062 1650548.02 +7924 1650202.40 +10076 1647970.24 +15859 1646036.28 +17932 1642640.66 +19694 1642089.50 +13827 1642001.31 +17963 1639689.00 +10698 1635848.26 +18003 1633530.78 +8416 1633366.77 +476 1631154.06 +2806 1630782.80 +12129 1628615.47 +11215 1626624.70 +14061 1624933.44 +5956 1623586.10 +9043 1622670.40 +13287 1621980.36 +11410 1621420.90 +13990 1621268.20 +12952 1619215.18 +15181 1619088.68 +9784 1618120.53 +10733 1616168.88 +16054 1614531.23 +5864 1614397.83 +1875 1611927.00 +17381 1611664.80 +14562 1607467.92 +575 1605941.73 +2005 1605591.72 +4332 1605448.83 +4653 1602596.30 +15403 1601830.44 +17430 1599681.42 +4798 1593630.50 +12991 1593321.52 +15653 1593138.66 +10066 1593049.06 +8892 1592100.90 +6708 1590159.12 +9825 1589403.92 +8271 1588475.41 +17084 1584280.88 +4003 1583631.00 +869 1582643.16 +16400 1582313.20 +19088 1581708.56 +6581 1581346.80 +9481 1581048.60 +6092 1580846.49 +3624 1578777.30 +6503 1578507.78 +14557 1578280.96 +2428 1577543.92 +15513 1573560.21 +4641 1573363.54 +10152 1570213.60 +5932 1566902.52 +7482 1561323.50 +13745 1558358.34 +2251 1558274.70 +9845 1558068.12 +7603 1557388.20 +1809 1553837.20 +18128 1547643.36 +8086 1543199.04 +14948 1541721.57 +16725 1540948.50 +2999 1540317.66 +8861 1540008.47 +1964 1538815.25 +19374 1537884.78 +15428 1535994.36 +7449 1534782.48 +16884 1534509.16 +10271 1534397.34 +11782 1529963.22 +8184 1529750.70 +4560 1527433.24 +4616 1525374.46 +3814 1524077.04 +17265 1523932.08 +16520 1522906.28 +10475 1518705.06 +5094 1517317.83 +8626 1515142.07 +19895 1512286.68 +19933 1506235.36 +6854 1505626.00 +13995 1505562.18 +7102 1504945.67 +9079 1501237.20 +18329 1500146.90 +3742 1496990.77 +12395 1496904.43 +12214 1496489.40 +12298 1495554.30 +4978 1495389.50 +2927 1494280.10 +2119 1494151.14 +15143 1492039.75 +14548 1487406.60 +840 1486128.98 +5902 1486097.28 +10614 1482144.72 +5895 1481356.80 +15958 1480951.60 +11408 1479948.96 +8407 1474236.00 +6243 1471007.85 +10389 1469004.46 +13871 1468938.64 +19811 1464597.09 +10495 1464290.49 +4389 1463010.83 +1311 1461703.36 +17874 1459408.88 +6597 1458761.87 +19211 1456741.63 +12879 1456178.24 +8840 1455731.46 +14755 1454890.60 +16957 1454465.96 +9257 1454388.76 +5193 1454011.32 +6884 1452474.60 +19948 1452024.00 +15076 1448395.00 +16016 1447557.45 +11693 1445839.68 +6975 1440516.96 +4290 1439768.46 +18900 1438722.10 +14383 1438477.92 +15098 1435941.78 +9322 1435282.80 +458 1433040.45 +10042 1432906.35 +5052 1431900.90 +6600 1431116.55 +3630 1428665.04 +9636 1428193.84 +16511 1427308.74 +4045 1427248.35 +19562 1426348.82 +8814 1425690.09 +2616 1425178.04 +4587 1425109.40 +148 1424237.37 +2712 1423780.26 +10863 1423386.16 +16096 1421942.09 +18936 1421938.65 +18327 1419872.92 +11620 1419050.10 +3740 1418609.85 +3457 1418603.50 +1185 1417637.47 +8178 1417357.26 +17791 1413293.13 +13608 1411323.12 +17849 1409613.50 +6814 1406228.40 +14022 1406138.04 +14231 1403771.52 +19546 1402854.60 +19619 1402389.16 +5609 1402302.54 +5342 1401567.59 +3084 1401096.10 +5708 1400334.90 +17998 1399862.45 +19850 1397630.33 +14004 1395443.10 +13071 1394653.24 +2797 1393747.58 +2866 1392947.25 +19809 1389067.68 +13600 1380865.80 +13614 1380654.36 +5884 1380319.74 +9404 1378623.66 +10656 1376954.32 +12324 1376502.40 +7325 1375030.43 +13295 1373987.34 +11864 1373555.68 +6987 1373481.51 +8386 1371854.41 +10916 1370374.32 +12867 1369058.11 +14668 1369040.34 +13383 1367342.30 +18572 1366953.96 +1152 1366861.38 +6015 1366452.18 +3344 1366185.15 +7889 1365521.92 +13345 1364088.88 +6276 1363421.62 +8069 1361824.20 +17509 1360892.49 +15137 1358678.07 +17163 1357391.52 +4704 1356692.40 +8609 1356578.19 +12644 1356088.14 +17141 1356022.38 +11805 1354826.78 +6386 1354187.22 +3004 1352173.44 +8634 1350211.80 +4399 1349881.20 +10362 1349411.34 +1572 1348835.20 +7359 1348224.10 +11884 1346696.82 +11671 1346424.15 +5350 1346359.28 +3119 1345996.48 +5307 1345356.00 +16117 1345045.12 +8715 1342665.72 +5398 1341179.28 +7627 1338820.56 +8457 1337714.68 +4958 1334732.71 +84 1334146.71 +6932 1333235.36 +757 1332921.07 +4076 1332441.00 +1751 1329112.32 +15701 1327052.16 +4119 1326549.90 +1562 1325604.28 +8741 1325517.60 +1135 1325422.71 +1002 1323418.65 +5832 1323085.71 +5368 1322793.96 +5382 1322628.84 +5616 1319082.26 +2832 1318691.95 +3895 1317858.44 +8629 1317756.51 +5709 1317058.68 +18383 1316451.05 +15797 1314806.64 +1900 1313660.40 +13882 1310455.86 +6785 1309877.80 +14855 1309280.76 +7761 1308602.24 +14268 1306810.40 +6257 1306056.96 +19002 1305509.52 +5095 1303729.02 +10320 1301657.58 +7826 1299561.68 +13359 1298717.14 +7436 1298127.36 +5644 1295055.77 +11327 1290526.41 +5277 1289329.65 +15932 1286235.84 +14322 1284809.36 +144 1284270.12 +3043 1281162.79 +16788 1280955.34 +17136 1280443.12 +12560 1279117.95 +13833 1278834.75 +5414 1277893.26 +12582 1277592.32 +4644 1277535.00 +14032 1277077.88 +18325 1271719.68 +7072 1271228.48 +16868 1267469.42 +8137 1267425.81 +5976 1266206.85 +14125 1265569.05 +13299 1265287.55 +18376 1264249.30 +6157 1261759.92 +5002 1261669.64 +13368 1260918.60 +15589 1260059.76 +2149 1258981.44 +9639 1256283.38 +11689 1256027.92 +9083 1245924.24 +16231 1242625.65 +5084 1242385.28 +11634 1240760.18 +15617 1239731.25 +9865 1237181.62 +14212 1236365.52 +10325 1235223.36 +19582 1235105.76 +740 1234746.81 +19231 1233623.10 +16840 1233063.85 +5703 1231744.33 +5761 1229435.20 +15630 1226611.62 +10408 1224698.40 +9177 1221942.51 +13389 1221666.75 +6104 1221577.92 +9673 1218826.64 +2707 1217124.48 +18672 1214208.80 +5112 1209590.20 +6264 1208318.50 +18496 1207881.75 +10971 1207183.52 +19059 1206729.90 +431 1205938.44 +3821 1201192.75 +826 1200454.62 +3317 1200440.90 +19689 1198899.52 +19641 1198797.99 +6379 1197195.50 +814 1194417.40 +18643 1194000.78 +11865 1193965.76 +12393 1193896.80 +9218 1193660.58 +8674 1191881.32 +8582 1191804.02 +13084 1191508.00 +18844 1190239.96 +16061 1189935.00 +6134 1185550.80 +8628 1183245.60 +8884 1181547.48 +7697 1181032.50 +9044 1180922.60 +13257 1180158.57 +8066 1178808.12 +5876 1177376.80 +14694 1177059.31 +16062 1175391.00 +9104 1175178.90 +11600 1175091.06 +10337 1172684.92 +19188 1172349.78 +8833 1171372.93 +6895 1170602.07 +14100 1168878.40 +13538 1168554.28 +3408 1166645.16 +1860 1165673.68 +13436 1164278.70 +19325 1162733.70 +7403 1161982.00 +4882 1161404.81 +13105 1161320.58 +17880 1161256.02 +19284 1160927.60 +13476 1159035.15 +18913 1158208.30 +18523 1158135.00 +12508 1157538.45 +9090 1156362.64 +17653 1154338.08 +3926 1152652.52 +10183 1148324.57 +7556 1146268.14 +16436 1142656.47 +4741 1141614.00 +15651 1141497.93 +3183 1140081.36 +9532 1139902.50 +16403 1139306.37 +2368 1137421.16 +3889 1136395.50 +2885 1135838.14 +7851 1135110.76 +16234 1135017.24 +12746 1134531.04 +2647 1132941.12 +5373 1132158.01 +10340 1132004.24 +8873 1131949.28 +1132 1131338.88 +15594 1131328.62 +4376 1130282.20 +240 1126682.48 +2231 1124447.15 +929 1121383.92 +11599 1119307.27 +3765 1119093.50 +17635 1118420.16 +7119 1118285.08 +15121 1117715.34 +11858 1116963.54 +16963 1116929.45 +16356 1113648.98 +6924 1112198.40 +16223 1111257.00 +18091 1110043.02 +12628 1108954.80 +16043 1108831.05 +9402 1108290.48 +708 1107084.00 +4078 1105993.96 +17593 1104713.40 +12776 1104362.59 +7583 1102813.53 +14619 1102675.80 +8842 1100110.26 +4196 1099726.55 +2019 1098178.64 +6863 1097246.36 +6489 1096503.07 +2459 1094813.04 +11964 1094485.02 +3236 1093969.80 +17647 1093809.15 +17648 1093114.62 +119 1092687.48 +9626 1092080.00 +9124 1091569.68 +13175 1089851.76 +2532 1088706.35 +16083 1088295.39 +8874 1086011.34 +12872 1082970.30 +19821 1082520.84 +4800 1080389.70 +18696 1079685.36 +19545 1079184.33 +13120 1077742.28 +10588 1076203.83 +17696 1075092.72 +14651 1073222.23 +903 1071146.76 +5858 1070259.48 +8302 1069504.80 +18728 1069225.51 +18026 1068569.00 +19383 1066907.58 +18690 1065930.90 +5924 1065143.12 +4880 1065011.75 +12439 1064381.19 +16529 1062371.70 +19653 1057683.56 +3136 1056810.44 +18932 1056193.65 +2124 1054160.52 +16851 1052646.84 +10123 1051624.00 +5618 1048447.93 +19851 1045187.85 +16278 1044808.38 +11479 1044276.22 +13263 1042046.20 +6041 1041123.38 +7193 1040455.32 +19408 1039430.01 +11260 1036828.52 +5179 1035633.44 +1331 1034398.00 +7706 1034249.40 +8436 1033549.35 +1801 1031886.00 +4170 1031642.90 +11827 1031139.39 +17114 1027985.88 +18278 1026583.11 +1995 1025165.68 +7667 1022980.15 +6559 1021635.45 +17488 1021612.13 +16059 1019781.19 +7633 1018782.57 +10032 1016809.50 +2899 1016438.76 +14628 1016033.20 +10126 1015846.78 +3884 1014413.50 +16913 1013604.40 +18644 1010288.10 +19870 1007919.36 +18564 1007416.20 +10179 1004920.00 +883 1004650.68 +3627 1004461.04 +18408 1002285.18 +8785 1001660.06 +18849 1000916.40 +12440 999894.16 +18516 999274.60 +5755 999117.99 +9739 998721.87 +16958 998513.05 +15352 996377.10 +9299 994737.10 +11553 994664.26 +2032 993403.95 +19125 992934.15 +7538 989526.15 +18943 988533.00 +8665 987149.00 +16459 986031.93 +368 985751.04 +4136 984763.65 +7827 984693.04 +3333 982245.38 +17809 980965.40 +932 980620.68 +16678 980499.91 +9964 977563.80 +14358 976236.28 +19715 974700.16 +4639 974392.24 +5628 974254.05 +13143 973719.03 +11088 972929.96 +110 968796.01 +16030 968498.40 +12092 967595.16 +19616 966431.34 +1847 965957.16 +13477 965756.25 +9186 962981.34 +6622 962153.52 +13043 962126.31 +6368 960535.16 +6557 960340.50 +11474 959540.40 +9755 959381.11 +4778 959120.16 +12616 956904.30 +18851 956179.56 +9107 954695.52 +271 954298.65 +19119 949928.75 +10375 947371.06 +7643 942287.22 +18582 941870.28 +14606 939281.15 +8483 939067.94 +3651 937217.96 +17607 936620.16 +5585 935459.98 +1630 932812.12 +7500 931442.75 +16824 929731.44 +13422 927685.44 +13801 926400.90 +18995 926105.52 +2334 924636.13 +13206 924175.17 +963 922402.66 +14908 922366.47 +4325 921271.44 +4886 920831.00 +7253 920276.00 +3751 919262.68 +16282 918933.00 +6302 918429.41 +13655 918324.00 +12316 914770.22 +3179 911620.50 +13438 909114.26 +11488 909044.62 +11361 908145.20 +10477 901784.58 +7365 900147.17 +19277 899900.44 +226 899761.38 +16287 899454.58 +4390 898878.63 +1550 898776.00 +9644 898445.10 +17330 897248.31 +12650 895000.48 +658 894649.92 +15284 892571.40 +19024 891979.20 +10377 891389.46 +7476 890710.19 +16253 890379.82 +4995 890107.44 +11770 885148.58 +18360 884180.88 +686 883878.60 +19181 882953.94 +18204 882570.92 +8644 881690.81 +4417 880993.50 +2633 880910.37 +6532 876322.50 +8081 871985.60 +3963 871550.90 +16327 871208.52 +9659 869041.50 +10599 868484.65 +3532 866357.38 +378 865911.54 +15502 865112.38 +17131 864977.08 +11367 862828.37 +2157 862828.16 +5603 861152.10 +10964 854847.21 +19888 850844.77 +10349 848868.44 +10257 846497.00 +9349 844048.48 +9609 843101.02 +14240 842825.54 +12333 842271.88 +7894 840906.68 +11625 840127.08 +255 838524.18 +13492 836492.80 +16386 831304.50 +13550 830651.94 +12610 829164.60 +1335 829070.50 +648 824028.10 +18645 823187.20 +6452 822401.70 +8774 821486.36 +2454 821166.88 +17060 817472.28 +18248 817428.48 +7272 816893.22 +16363 814467.03 +11972 810825.98 +4386 810015.90 +4841 807959.03 +9157 807780.54 +7444 803362.54 +19253 801097.75 +17059 801055.18 +7518 800781.60 +8070 800485.84 +3779 794856.70 +9303 794784.96 +3819 794693.52 +11473 794457.60 +2539 793908.97 +18664 792838.16 +18789 790057.45 +16474 787600.60 +18115 787415.90 +2075 784851.84 +19375 783701.50 +19202 782566.10 +17004 782347.68 +14088 781780.60 +2823 780046.65 +9445 779740.06 +2370 778730.55 +2345 778411.80 +5643 777348.60 +11521 776828.36 +19830 776390.30 +2767 775894.47 +6328 775090.58 +6119 775083.00 +15458 774661.78 +13840 774443.00 +14648 771094.72 +5677 768204.36 +12553 767871.26 +2677 767754.05 +14120 764268.75 +5349 764223.46 +8052 763781.00 +12912 763767.20 +15591 763521.28 +11386 762604.92 +15365 762590.40 +5586 760490.64 +5319 760122.22 +14419 759407.00 +10897 758587.68 +8013 756712.46 +18467 756137.40 +10339 755974.38 +1051 755393.58 +18808 754670.07 +10672 752363.08 +10752 751872.00 +18611 751215.30 +868 748493.06 +16084 745210.10 +19340 744998.04 +8876 744681.90 +4147 743885.81 +4122 743555.20 +8852 741043.03 +14893 740065.50 +11918 739955.90 +3502 739633.11 +6227 739209.50 +9187 737922.40 +18280 737507.19 +8899 736686.93 +10253 736572.34 +4140 734955.42 +19878 732123.81 +6739 732000.01 +7261 731959.62 +17110 731579.16 +17194 730970.35 +12595 730734.38 +9850 728298.60 +18996 725894.76 +6997 721063.20 +5669 720956.70 +2184 720528.00 +16987 718963.00 +15608 718666.56 +10553 718099.20 +3134 717103.30 +7070 715509.60 +17645 714081.64 +12121 713814.61 +1279 710088.75 +11779 709146.25 +6621 708524.64 +16597 708244.95 +7146 706965.75 +17518 706355.43 +4154 704086.88 +16090 702130.20 +16610 700366.38 +13823 700025.10 +11470 699696.34 +8269 699644.41 +18368 697225.98 +16359 696749.04 +4562 696027.81 +11530 695781.24 +10888 695332.44 +7840 694064.78 +18285 692323.10 +17821 691327.80 +4509 690905.52 +1107 687277.80 +17600 687171.00 +7326 686379.12 +4649 685479.20 +11106 684800.00 +10586 683696.23 +16850 682672.32 +11648 682541.48 +11010 682487.84 +13821 682057.67 +18140 681645.48 +16944 681058.49 +16476 680013.45 +11084 677803.74 +15616 675492.48 +15052 675143.37 +18604 675065.59 +16157 672871.84 +17586 672865.92 +3586 670977.06 +17383 670781.64 +11211 667600.92 +9553 667158.47 +10805 666257.56 +18159 664172.34 +7408 661874.24 +8084 660284.42 +17043 659951.76 +847 659530.18 +13797 657922.80 +5857 657834.09 +11886 657353.69 +19740 656545.07 +12043 655994.50 +19263 654466.45 +18592 649935.00 +7974 649679.32 +6607 649092.21 +634 648192.94 +16551 646627.52 +16211 645862.80 +4095 643986.24 +1114 639657.04 +5383 638309.28 +8298 637017.60 +13123 632410.16 +3052 631869.15 +8091 630857.50 +9922 630642.88 +4791 630131.33 +1436 629369.86 +1426 628241.25 +17295 627959.18 +3863 627598.28 +97 626592.12 +15892 624640.41 +17350 624434.91 +11771 623709.36 +3395 623076.84 +552 622856.59 +8121 620644.50 +14380 620623.36 +10480 618647.20 +1792 617391.32 +7962 615206.66 +5446 615127.73 +7224 615007.80 +14166 614256.84 +16817 614072.60 +5298 613673.10 +15562 612427.80 +16197 611744.40 +7835 611155.54 +7187 611130.96 +18377 609733.31 +7644 609672.00 +645 607985.84 +12539 607267.10 +4568 606708.90 +19353 605121.65 +3157 605048.80 +12790 604517.76 +18459 604378.11 +13685 603255.06 +9817 603241.35 +11126 602064.54 +19538 601517.85 +17179 600665.60 +10134 600237.12 +4875 599774.70 +6923 599427.66 +17902 598573.30 +16489 596732.64 +1883 596048.73 +15960 596045.79 +12366 595898.01 +15310 594751.85 +3860 593622.12 +16799 592959.15 +10628 592922.98 +7884 592282.08 +13025 591341.65 +10052 589303.44 +2386 586754.55 +5466 586707.72 +5327 584317.50 +302 582503.76 +14684 581500.79 +3904 580408.08 +17550 580311.93 +10115 578872.02 +10436 578429.00 +14227 577834.64 +15597 577684.50 +15530 576776.10 +4495 575799.12 +19015 573844.05 +19446 573115.29 +521 572606.30 +8000 571204.26 +11075 571052.58 +5369 569455.60 +19359 567754.01 +15147 567201.42 +11623 566420.40 +18540 565692.19 +17147 565595.20 +4385 565432.86 +10102 564403.20 +16308 563740.51 +14858 563180.20 +9222 561328.11 +15434 561081.78 +18353 561052.80 +13323 559636.05 +4134 559387.86 +16121 558160.47 +4903 556776.00 +17776 554339.18 +16408 552944.70 +19746 552381.44 +12885 551994.07 +1615 550834.35 +10235 550447.74 +7755 549525.60 +5488 547176.06 +2884 545432.91 +17377 543375.96 +18653 542456.46 +4201 542010.07 +15811 540814.40 +6413 539882.49 +4306 538725.12 +2780 538546.33 +15945 536488.40 +14461 535327.96 +8966 535307.08 +4878 534848.32 +2131 534687.16 +3930 533828.24 +2408 533672.94 +16102 532910.75 +18228 532842.45 +11271 532651.24 +682 531371.70 +14993 530208.30 +19179 529993.60 +8218 528876.80 +13558 528740.55 +3458 528294.91 +3300 526425.20 +13731 525692.05 +7132 525692.01 +8960 524338.56 +711 523794.18 +11914 523793.18 +19404 522622.62 +19912 522064.92 +17356 519632.12 +15231 518574.29 +3622 518021.78 +7440 515730.81 +19648 515645.52 +13859 514836.40 +17954 514679.55 +3996 514030.56 +1823 513844.38 +16740 512589.20 +3642 510694.38 +3133 508520.32 +10840 507788.99 +13271 506252.70 +19845 506037.70 +1596 505225.82 +6877 504480.48 +16648 502701.90 +6072 501742.52 +14384 501514.09 +7275 501492.54 +10816 500849.70 +19783 499605.40 +13727 499594.55 +19379 498554.14 +13619 497012.62 +15649 495023.75 +827 494884.18 +8037 494712.99 +4817 491132.27 +12032 491090.02 +14989 490884.31 +15267 490589.58 +10216 489859.59 +19464 489076.16 +5076 488878.77 +18562 486103.56 +19151 484434.15 +13687 484196.36 +14532 484177.52 +12563 483576.72 +9217 483259.20 +6348 481950.56 +15248 481809.24 +19884 479916.06 +2929 479758.02 +16431 479482.38 +18627 479153.92 +11995 479067.84 +10147 478475.86 +18191 477161.25 +10354 476558.64 +16871 476466.90 +9147 475695.83 +19615 475611.12 +7964 475007.73 +9296 474259.59 +9534 473728.50 +5963 473472.78 +8339 472930.18 +2208 469865.25 +1338 469668.60 +9629 469466.79 +8615 469072.31 +1589 468032.32 +1142 467469.62 +11398 466612.17 +14524 465466.71 +18343 465072.51 +12174 462715.60 +5113 461798.42 +3964 460889.71 +5072 460641.75 +19532 457296.90 +16368 457068.50 +13327 455568.75 +3498 455507.80 +14515 454234.20 +2376 453831.46 +13842 453709.80 +17817 453362.07 +9581 453321.00 +6874 453007.30 +6152 452603.80 +2528 451368.75 +12445 451065.42 +14286 450652.80 +8046 449859.69 +13743 449663.11 +16881 447039.65 +4624 446875.40 +17039 446598.24 +9558 446209.92 +14333 446137.12 +1147 445357.16 +4963 444298.02 +16764 442600.66 +3130 440642.37 +12291 438361.53 +10616 438327.65 +1891 438161.50 +1869 435965.60 +7791 433569.14 +7862 433004.81 +7882 432844.10 +13856 432348.94 +17567 432067.60 +3047 431188.28 +15462 430794.52 +18940 430547.44 +5743 430353.20 +6901 430326.95 +394 430080.02 +5574 429849.12 +13669 429391.05 +19695 429200.28 +633 428209.60 +13052 427656.16 +4108 427362.54 +18958 425107.95 +7648 422904.24 +8602 422748.26 +2905 422636.00 +10627 421632.81 +15023 419740.52 +13288 419592.39 +10293 419449.95 +15791 419040.72 +7795 419012.49 +2211 418460.16 +12106 417856.40 +17380 417316.70 +16991 417144.00 +13076 416336.80 +5506 415340.80 +19302 414824.60 +7 414558.20 +2140 414435.89 +3897 414162.00 +5386 410059.52 +9386 409776.96 +713 409772.48 +5542 409678.50 +10178 408933.55 +18318 408350.16 +4135 408307.28 +9740 408248.88 +3223 407602.20 +9076 405293.90 +4486 403769.37 +16605 403305.90 +2586 403287.42 +13392 401370.79 +13122 401355.20 +6878 399595.46 +2711 399275.36 +16389 398905.00 +11428 397763.10 +12211 397286.64 +14211 397003.26 +1404 396559.68 +5106 395650.50 +14586 394990.50 +12073 394220.80 +18354 393073.92 +4182 391917.32 +17944 391714.40 +6275 389733.70 +12028 389573.04 +17283 388783.26 +3472 388739.22 +16320 388217.28 +90 387858.36 +8895 387658.53 +4374 386101.42 +8854 385400.73 +12528 384380.55 +16942 384250.64 +13365 384080.16 +13408 382965.68 +9898 379105.20 +755 378498.86 +6308 378400.20 +5267 376808.10 +7370 376441.28 +4866 375603.44 +8125 373807.14 +17013 371955.40 +17623 371460.60 +14116 371280.00 +12084 370901.37 +13579 370160.07 +5032 366346.70 +15858 365016.32 +5131 364325.78 +5403 363094.14 +5448 362910.80 +7481 361264.08 +52 358792.36 +15163 358417.28 +9555 358332.03 +253 357995.52 +13948 356288.85 +19347 356009.50 +3389 355636.58 +10367 354743.94 +1346 350637.43 +12603 349022.40 +408 348939.75 +7578 346501.12 +8370 344853.10 +5892 343870.80 +11846 340996.68 +3349 339738.54 +15627 339072.90 +15363 338539.05 +4822 337689.00 +12325 333570.38 +12087 333550.20 +4621 331292.60 +2615 330807.95 +2740 328308.60 +13839 326190.62 +6520 325524.13 +1617 324231.60 +14387 323321.01 +16629 322145.10 +15575 321494.32 +16964 320794.80 +18586 320402.10 +19403 317039.36 +15056 316535.25 +9369 315714.56 +14779 315580.65 +7954 315520.00 +8823 315432.36 +11932 315011.13 +8958 314998.08 +8586 313654.90 +19932 312586.56 +17368 311944.36 +4129 311286.40 +2727 308645.55 +18036 308176.07 +4257 306189.37 +4824 305727.80 +16618 303036.32 +15389 302119.30 +6796 301998.23 +13966 301049.76 +6130 297628.80 +5287 295215.36 +10252 294329.52 +6604 293585.94 +16622 292751.20 +5860 291985.00 +5070 291426.80 +14917 291102.52 +13432 290223.75 +11896 289787.19 +4372 288520.05 +12801 288085.60 +9736 287393.07 +13764 286449.12 +12349 285645.15 +5839 283663.52 +14007 282710.52 +2959 281456.00 +1225 280769.77 +5638 280292.42 +18755 280252.80 +17990 279389.35 +10659 278245.52 +11963 278048.94 +2076 275765.05 +14964 275425.00 +5840 275369.70 +15667 275267.20 +17691 275081.12 +6074 274454.46 +5167 274401.90 +4630 273177.90 +18050 272915.86 +15644 271246.56 +3309 271045.34 +2253 269714.04 +17083 268831.20 +17286 268754.01 +15138 268457.07 +5605 268349.20 +464 268085.70 +964 267485.46 +4714 267333.15 +12707 266687.50 +1632 266143.26 +7875 265152.36 +7139 265131.15 +14729 263650.45 +14356 261900.10 +18010 260543.58 +13406 260533.98 +14094 258814.57 +17677 258768.40 +11839 258542.83 +15795 257288.40 +7958 253638.00 +17548 253450.73 +18632 250603.04 +14883 249422.25 +13648 248378.20 +17765 248336.06 +3675 246654.92 +11697 245964.42 +3205 245462.25 +6963 244471.50 +2995 243620.88 +4597 243553.80 +10704 243357.48 +13243 243087.86 +4622 242676.00 +14171 242390.33 +6112 241420.32 +1628 239833.44 +1960 239649.93 +6306 238378.14 +12770 238264.67 +7226 237094.09 +21 236941.20 +11755 236331.65 +3641 236093.94 +7969 233819.20 +14430 233365.35 +8700 232934.00 +9793 231812.88 +3526 230570.30 +1928 230522.71 +11603 229962.03 +14794 229545.84 +1060 229364.52 +7557 228913.92 +10691 228843.42 +12336 228819.28 +17082 227969.07 +4755 227467.50 +14342 226880.72 +16876 226782.32 +17424 226333.80 +17355 225495.86 +6330 225197.00 +10444 224726.04 +6707 224680.56 +9713 224329.50 +7620 224175.08 +2867 220983.48 +12657 219172.40 +15339 218512.90 +5257 218373.69 +8383 215793.84 +10231 215061.30 +15086 214998.14 +11134 214758.90 +13586 214473.87 +19110 213151.47 +2814 211548.96 +8889 211100.67 +17326 210727.24 +9118 210289.00 +2743 210220.68 +6181 208024.75 +8190 207525.78 +3673 204191.52 +12729 204122.25 +1609 200863.96 +11368 200296.50 +10532 199656.06 +19725 198464.00 +10783 197926.56 +8779 197640.39 +7900 197577.62 +11364 194362.56 +10393 194167.20 +19076 193925.22 +11066 193589.72 +13593 193248.00 +133 193184.72 +10755 193116.40 +4419 192652.97 +8016 192357.48 +7815 189704.46 +8920 189388.30 +15869 188839.86 +18168 188835.84 +17088 188770.68 +12467 188733.80 +7192 188476.40 +8527 187128.12 +13644 186805.80 +639 186533.28 +7860 186430.08 +907 186159.89 +10067 185022.63 +13231 184250.52 +889 181904.31 +19115 181524.24 +2179 180659.85 +1242 179182.38 +4648 176590.70 +14368 176065.00 +19520 174676.26 +15740 173575.42 +9693 172042.27 +10157 170047.50 +15368 169315.12 +1349 168440.81 +9998 168255.12 +11149 166590.27 +9383 165160.92 +17986 162746.85 +6110 162588.24 +10869 162063.25 +12074 161659.14 +11389 161438.37 +5408 161423.88 +15586 161401.40 +9625 161061.93 +2791 161036.20 +7327 160098.20 +14139 159801.92 +19372 158688.45 +4450 158315.40 +4884 157226.88 +4133 157004.55 +3803 156503.64 +936 154045.80 +10523 153396.99 +4476 152675.00 +11125 152087.04 +10034 150861.42 +13386 149003.32 +18812 148899.28 +8181 147472.17 +16210 147247.30 +6562 146836.83 +436 146563.28 +1032 146408.40 +3032 144144.84 +18532 143103.20 +8758 142770.87 +8442 142071.60 +9231 142059.48 +13062 141412.12 +10600 141334.16 +19099 139067.10 +3055 138131.33 +3420 138030.24 +13349 137588.19 +7296 136659.20 +9661 136565.95 +8597 136552.65 +3587 136393.38 +3325 136229.73 +14332 134995.48 +14644 131902.89 +7872 131897.81 +19938 131830.26 +3613 130052.52 +19576 130044.38 +10473 129795.12 +3717 128415.37 +17101 127762.30 +860 127745.62 +12062 127627.62 +7068 127214.37 +12958 126118.24 +7018 126117.00 +10077 125380.40 +5389 125161.16 +18092 124733.94 +11077 124640.00 +19790 123414.00 +19257 122563.20 +1707 122400.00 +13064 120777.60 +8551 119975.59 +14852 118954.44 +13918 118924.20 +11954 118118.44 +9831 116576.75 +15433 116133.11 +16666 115973.47 +1636 114972.84 +2396 114097.20 +13353 113947.96 +1368 111906.63 +19350 111835.02 +6483 111282.85 +6351 111240.78 +9920 111216.05 +2830 110296.68 +14103 110276.64 +12830 108360.72 +18315 107893.98 +8040 107439.92 +2628 107236.80 +12949 107146.50 +10782 106793.02 +19078 106483.41 +19457 105913.35 +19787 104845.52 +4017 104569.30 +15067 104369.02 +586 103845.50 +9622 103326.30 +7281 101198.40 +6745 99053.85 +13051 98731.68 +6558 98153.28 +1729 97155.24 +9753 97004.84 +7845 95723.81 +5500 95642.56 +6360 95203.36 +17553 95037.62 +7129 94479.40 +3354 93871.80 +16562 93663.62 +5964 93171.92 +8345 92505.60 +11635 92318.94 +12690 91550.08 +10491 91345.33 +18084 91087.20 +5422 90654.09 +3081 90370.98 +3360 90031.95 +17275 89690.96 +8349 89400.09 +10143 89354.98 +4084 88330.88 +15330 87562.98 +1389 87438.91 +1476 86896.04 +9972 86497.23 +5198 86195.20 +14052 85855.68 +6132 85585.07 +7083 85162.52 +18598 85085.04 +681 84984.31 +13532 83267.60 +9434 82294.75 +16616 81461.64 +17588 81373.20 +6787 80627.24 +11867 78462.65 +14703 78175.50 +9994 78053.47 +2049 78016.40 +7410 77985.70 +16766 76765.40 +11157 76738.06 +1299 76722.60 +19207 75180.30 +16915 75013.02 +16413 74790.56 +19693 74245.08 +11554 73640.57 +6958 72982.56 +12141 72783.72 +18847 72720.72 +10868 72465.89 +5851 72159.06 +6645 71811.18 +15327 71625.60 +9380 71440.00 +15002 70963.20 +12858 70668.78 +18832 69991.50 +16381 69607.00 +17731 66137.05 +14958 66044.20 +577 65929.70 +3386 65797.19 +17602 65745.90 +7775 65371.28 +12653 64919.68 +1382 64638.42 +17028 64470.01 +6856 63921.78 +11644 62988.75 +10751 62470.74 +12403 61784.18 +3138 61465.80 +11180 60953.10 +5157 60481.26 +15045 60216.48 +11823 59405.72 +9211 58583.86 +14479 58195.06 +19053 57562.73 +1365 57470.40 +1095 57406.08 +19486 56736.02 +1958 56616.56 +11373 55984.56 +2156 55692.00 +18507 55231.80 +17498 54981.68 +19514 54231.24 +64 53189.28 +16179 52644.50 +12648 52342.68 +15833 52164.75 +15388 51211.27 +17398 50793.28 +17253 50621.76 +16147 50376.48 +12478 49894.02 +314 48762.30 +8953 48242.73 +11338 47212.84 +4211 47028.88 +2821 46928.90 +7300 46427.74 +13799 46054.97 +7532 46027.50 +3981 45978.66 +1541 45576.00 +3537 44017.40 +15821 43800.00 +18389 43748.43 +14312 43276.30 +2425 43175.84 +17169 42888.56 +5253 42329.15 +4854 42251.67 +19626 41949.41 +17058 41896.50 +5701 41342.40 +372 41157.76 +14271 41015.90 +16574 40958.50 +4671 40184.10 +10096 39102.25 +16647 38931.42 +10569 38663.10 +14535 38621.26 +14725 38535.24 +9606 38215.68 +19225 37457.30 +17327 37077.12 +5199 36498.24 +186 36468.75 +16854 35525.73 +15992 35492.00 +19807 34632.12 +11076 34028.88 +15641 33618.62 +11448 33411.52 +481 33101.80 +2984 33087.62 +13330 33085.80 +7257 32965.12 +10399 31781.75 +12811 31659.00 +19591 31213.82 +1723 30638.38 +16112 30103.15 +13994 29299.42 +2001 28345.00 +15622 28242.00 +3122 27982.50 +19789 27788.32 +10128 26191.32 +17206 25627.50 +8328 25540.56 +13950 25454.57 +6553 25389.70 +4957 25045.76 +19384 25028.84 +461 24725.22 +14821 24639.15 +16932 24418.47 +1428 23872.42 +15854 23789.28 +18081 23291.26 +10921 23263.24 +4073 22698.90 +2776 22492.19 +19425 21753.48 +14611 21369.15 +17542 20932.38 +1238 20893.18 +386 20884.24 +13560 20879.84 +19147 20462.27 diff --git a/regression-test/data/nereids_tpch_p0/tpch/q14.out b/regression-test/data/nereids_tpch_p0/tpch/q14.out index 90897ced93..300b74fc25 100644 --- a/regression-test/data/nereids_tpch_p0/tpch/q14.out +++ b/regression-test/data/nereids_tpch_p0/tpch/q14.out @@ -1,7 +1,7 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -16.283855700 +16.283855 -- !select -- -16.283855700 +16.283855 diff --git a/regression-test/data/nereids_tpch_p0/tpch/q15.out b/regression-test/data/nereids_tpch_p0/tpch/q15.out index fe76ded6ab..3c2d8798b8 100644 --- a/regression-test/data/nereids_tpch_p0/tpch/q15.out +++ b/regression-test/data/nereids_tpch_p0/tpch/q15.out @@ -1,7 +1,7 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -677 Supplier#000000677 8mhrffG7D2WJBSQbOGstQ 23-290-639-3315 1614410.292800000 +677 Supplier#000000677 8mhrffG7D2WJBSQbOGstQ 23-290-639-3315 1614410.2928 -- !select -- -677 Supplier#000000677 8mhrffG7D2WJBSQbOGstQ 23-290-639-3315 1614410.292800000 +677 Supplier#000000677 8mhrffG7D2WJBSQbOGstQ 23-290-639-3315 1614410.2928 diff --git a/regression-test/data/nereids_tpch_p0/tpch/q17.out b/regression-test/data/nereids_tpch_p0/tpch/q17.out index a0d5f8611f..f82000e372 100644 --- a/regression-test/data/nereids_tpch_p0/tpch/q17.out +++ b/regression-test/data/nereids_tpch_p0/tpch/q17.out @@ -1,7 +1,7 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -23512.752857143 +23512.75 -- !select -- -23512.752857143 +23512.75 diff --git a/regression-test/data/nereids_tpch_p0/tpch/q18.out b/regression-test/data/nereids_tpch_p0/tpch/q18.out index 2a81f8dea4..cb873122bd 100644 --- a/regression-test/data/nereids_tpch_p0/tpch/q18.out +++ b/regression-test/data/nereids_tpch_p0/tpch/q18.out @@ -1,15 +1,15 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -Customer#000001639 1639 502886 1994-04-12 456423.880000000 312.000000000 -Customer#000006655 6655 29158 1995-10-21 452805.020000000 305.000000000 -Customer#000014110 14110 565574 1995-09-24 425099.850000000 301.000000000 -Customer#000001775 1775 6882 1997-04-09 408368.100000000 303.000000000 -Customer#000011459 11459 551136 1993-05-19 386812.740000000 308.000000000 +Customer#000001639 1639 502886 1994-04-12 456423.88 312.00 +Customer#000006655 6655 29158 1995-10-21 452805.02 305.00 +Customer#000014110 14110 565574 1995-09-24 425099.85 301.00 +Customer#000001775 1775 6882 1997-04-09 408368.10 303.00 +Customer#000011459 11459 551136 1993-05-19 386812.74 308.00 -- !select -- -Customer#000001639 1639 502886 1994-04-12 456423.880000000 312.000000000 -Customer#000006655 6655 29158 1995-10-21 452805.020000000 305.000000000 -Customer#000014110 14110 565574 1995-09-24 425099.850000000 301.000000000 -Customer#000001775 1775 6882 1997-04-09 408368.100000000 303.000000000 -Customer#000011459 11459 551136 1993-05-19 386812.740000000 308.000000000 +Customer#000001639 1639 502886 1994-04-12 456423.88 312.00 +Customer#000006655 6655 29158 1995-10-21 452805.02 305.00 +Customer#000014110 14110 565574 1995-09-24 425099.85 301.00 +Customer#000001775 1775 6882 1997-04-09 408368.10 303.00 +Customer#000011459 11459 551136 1993-05-19 386812.74 308.00 diff --git a/regression-test/data/nereids_tpch_p0/tpch/q19.out b/regression-test/data/nereids_tpch_p0/tpch/q19.out index 2e19a4100d..7c5431f987 100644 --- a/regression-test/data/nereids_tpch_p0/tpch/q19.out +++ b/regression-test/data/nereids_tpch_p0/tpch/q19.out @@ -1,7 +1,7 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -168597.286000000 +168597.2860 -- !select -- -168597.286000000 +168597.2860 diff --git a/regression-test/data/nereids_tpch_p0/tpch/q2.out b/regression-test/data/nereids_tpch_p0/tpch/q2.out index 84d062239e..836683f0a3 100644 --- a/regression-test/data/nereids_tpch_p0/tpch/q2.out +++ b/regression-test/data/nereids_tpch_p0/tpch/q2.out @@ -1,93 +1,93 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -9828.210000000 Supplier#000000647 UNITED KINGDOM 13120 Manufacturer#5 x5U7MBZmwfG9 33-258-202-4782 s the slyly even ideas poach fluffily -9508.370000000 Supplier#000000070 FRANCE 3563 Manufacturer#1 INWNH2w,OOWgNDq0BRCcBwOMQc6PdFDc4 16-821-608-1166 ests sleep quickly express ideas. ironic ideas haggle about the final T -9508.370000000 Supplier#000000070 FRANCE 17268 Manufacturer#4 INWNH2w,OOWgNDq0BRCcBwOMQc6PdFDc4 16-821-608-1166 ests sleep quickly express ideas. ironic ideas haggle about the final T -9453.010000000 Supplier#000000802 ROMANIA 10021 Manufacturer#5 ,6HYXb4uaHITmtMBj4Ak57Pd 29-342-882-6463 gular frets. permanently special multipliers believe blithely alongs -9453.010000000 Supplier#000000802 ROMANIA 13275 Manufacturer#4 ,6HYXb4uaHITmtMBj4Ak57Pd 29-342-882-6463 gular frets. permanently special multipliers believe blithely alongs -9192.100000000 Supplier#000000115 UNITED KINGDOM 13325 Manufacturer#1 nJ 2t0f7Ve,wL1,6WzGBJLNBUCKlsV 33-597-248-1220 es across the carefully express accounts boost caref -9032.150000000 Supplier#000000959 GERMANY 4958 Manufacturer#4 8grA EHBnwOZhO 17-108-642-3106 nding dependencies nag furiou -8702.020000000 Supplier#000000333 RUSSIA 11810 Manufacturer#3 MaVf XgwPdkiX4nfJGOis8Uu2zKiIZH 32-508-202-6136 oss the deposits cajole carefully even pinto beans. regular foxes detect alo -8615.500000000 Supplier#000000812 FRANCE 10551 Manufacturer#2 8qh4tezyScl5bidLAysvutB,,ZI2dn6xP 16-585-724-6633 y quickly regular deposits? quickly pending packages after the caref -8615.500000000 Supplier#000000812 FRANCE 13811 Manufacturer#4 8qh4tezyScl5bidLAysvutB,,ZI2dn6xP 16-585-724-6633 y quickly regular deposits? quickly pending packages after the caref -8488.530000000 Supplier#000000367 RUSSIA 6854 Manufacturer#4 E Sv9brQVf43Mzz 32-458-198-9557 ages. carefully final excuses nag finally. carefully ironic deposits abov -8430.520000000 Supplier#000000646 FRANCE 11384 Manufacturer#3 IUzsmT,2oBgjhWP2TlXTL6IkJH,4h,1SJRt 16-601-220-5489 ites among the always final ideas kindle according to the theodolites. notornis in -8271.390000000 Supplier#000000146 RUSSIA 4637 Manufacturer#5 rBDNgCr04x0sfdzD5,gFOutCiG2 32-792-619-3155 s cajole quickly special requests. quickly enticing theodolites h -8096.980000000 Supplier#000000574 RUSSIA 323 Manufacturer#4 2O8 sy9g2mlBOuEjzj0pA2pevk, 32-866-246-8752 ully after the regular requests. slyly final dependencies wake slyly along the busy deposit -7392.780000000 Supplier#000000170 UNITED KINGDOM 7655 Manufacturer#2 RtsXQ,SunkA XHy9 33-803-340-5398 ake carefully across the quickly -7205.200000000 Supplier#000000477 GERMANY 10956 Manufacturer#5 VtaNKN5Mqui5yh7j2ldd5waf 17-180-144-7991 excuses wake express deposits. furiously careful asymptotes according to the carefull -6820.350000000 Supplier#000000007 UNITED KINGDOM 13217 Manufacturer#5 s,4TicNGB4uO6PaSqNBUq 33-990-965-2201 s unwind silently furiously regular courts. final requests are deposits. requests wake quietly blit -6721.700000000 Supplier#000000954 FRANCE 4191 Manufacturer#3 P3O5p UFz1QsLmZX 16-537-341-8517 ect blithely blithely final acco -6329.900000000 Supplier#000000996 GERMANY 10735 Manufacturer#2 Wx4dQwOAwWjfSCGupfrM 17-447-811-3282 ironic forges cajole blithely agai -6173.870000000 Supplier#000000408 RUSSIA 18139 Manufacturer#1 qcor1u,vJXAokjnL5,dilyYNmh 32-858-724-2950 blithely pending packages cajole furiously slyly pending notornis. slyly final -5364.990000000 Supplier#000000785 RUSSIA 13784 Manufacturer#4 W VkHBpQyD3qjQjWGpWicOpmILFehmEdWy67kUGY 32-297-653-2203 packages boost carefully. express ideas along -5069.270000000 Supplier#000000328 GERMANY 16327 Manufacturer#1 SMm24d WG62 17-231-513-5721 he unusual ideas. slyly final packages a -4941.880000000 Supplier#000000321 ROMANIA 7320 Manufacturer#5 pLngFl5yeMcHyov 29-573-279-1406 y final requests impress s -4672.250000000 Supplier#000000239 RUSSIA 12238 Manufacturer#1 XO101kgHrJagK2FL1U6QCaTE ncCsMbeuTgK6o8 32-396-654-6826 arls wake furiously deposits. even, regular depen -4586.490000000 Supplier#000000680 RUSSIA 5679 Manufacturer#3 UhvDfdEfJh,Qbe7VZb8uSGO2TU 0jEa6nXZXE 32-522-382-1620 the regularly regular dependencies. carefully bold excuses under th -4518.310000000 Supplier#000000149 FRANCE 18344 Manufacturer#5 pVyWsjOidpHKp4NfKU4yLeym 16-660-553-2456 ts detect along the foxes. final Tiresias are. idly pending deposits haggle; even, blithe pin -4315.150000000 Supplier#000000509 FRANCE 18972 Manufacturer#2 SF7dR8V5pK 16-298-154-3365 ronic orbits are furiously across the requests. quickly express ideas across the special, bold -3526.530000000 Supplier#000000553 FRANCE 8036 Manufacturer#4 a,liVofXbCJ 16-599-552-3755 lar dinos nag slyly brave -3526.530000000 Supplier#000000553 FRANCE 17018 Manufacturer#3 a,liVofXbCJ 16-599-552-3755 lar dinos nag slyly brave -3294.680000000 Supplier#000000350 GERMANY 4841 Manufacturer#4 KIFxV73eovmwhh 17-113-181-4017 e slyly special foxes. furiously unusual deposits detect carefully carefully ruthless foxes. quick -2972.260000000 Supplier#000000016 RUSSIA 1015 Manufacturer#4 YjP5C55zHDXL7LalK27zfQnwejdpin4AMpvh 32-822-502-4215 ously express ideas haggle quickly dugouts? fu -2963.090000000 Supplier#000000840 ROMANIA 3080 Manufacturer#2 iYzUIypKhC0Y 29-781-337-5584 eep blithely regular dependencies. blithely regular platelets sublate alongside o -2221.250000000 Supplier#000000771 ROMANIA 13981 Manufacturer#2 lwZ I15rq9kmZXUNhl 29-986-304-9006 nal foxes eat slyly about the fluffily permanent id -1381.970000000 Supplier#000000104 FRANCE 18103 Manufacturer#3 Dcl4yGrzqv3OPeRO49bKh78XmQEDR7PBXIs0m 16-434-972-6922 gular ideas. bravely bold deposits haggle through the carefully final deposits. slyly unusual idea -906.070000000 Supplier#000000138 ROMANIA 8363 Manufacturer#4 utbplAm g7RmxVfYoNdhcrQGWuzRqPe0qHSwbKw 29-533-434-6776 ickly unusual requests cajole. accounts above the furiously special excuses -765.690000000 Supplier#000000799 RUSSIA 11276 Manufacturer#2 jwFN7ZB3T9sMF 32-579-339-1495 nusual requests. furiously unusual epitaphs integrate. slyly -727.890000000 Supplier#000000470 ROMANIA 6213 Manufacturer#3 XckbzsAgBLbUkdfjgJEPjmUMTM8ebSMEvI 29-165-289-1523 gular excuses. furiously regular excuses sleep slyly caref -683.070000000 Supplier#000000651 RUSSIA 4888 Manufacturer#4 oWekiBV6s,1g 32-181-426-4490 ly regular requests cajole abou -167.560000000 Supplier#000000290 FRANCE 2037 Manufacturer#1 6Bk06GVtwZaKqg01 16-675-286-5102 the theodolites. ironic, ironic deposits above -91.390000000 Supplier#000000949 UNITED KINGDOM 9430 Manufacturer#2 a,UE,6nRVl2fCphkOoetR1ajIzAEJ1Aa1G1HV 33-332-697-2768 pinto beans. carefully express requests hagg --314.060000000 Supplier#000000510 ROMANIA 17242 Manufacturer#4 VmXQl ,vY8JiEseo8Mv4zscvNCfsY 29-207-852-3454 bold deposits. carefully even d --820.890000000 Supplier#000000409 GERMANY 2156 Manufacturer#5 LyXUYFz7aXrvy65kKAbTatGzGS,NDBcdtD 17-719-517-9836 y final, slow theodolites. furiously regular req --845.440000000 Supplier#000000704 ROMANIA 9926 Manufacturer#5 hQvlBqbqqnA5Dgo1BffRBX78tkkRu 29-300-896-5991 ctions. carefully sly requ --942.730000000 Supplier#000000563 GERMANY 5797 Manufacturer#1 Rc7U1cRUhYs03JD 17-108-537-2691 slyly furiously final decoys; silent, special realms poach f +9828.21 Supplier#000000647 UNITED KINGDOM 13120 Manufacturer#5 x5U7MBZmwfG9 33-258-202-4782 s the slyly even ideas poach fluffily +9508.37 Supplier#000000070 FRANCE 3563 Manufacturer#1 INWNH2w,OOWgNDq0BRCcBwOMQc6PdFDc4 16-821-608-1166 ests sleep quickly express ideas. ironic ideas haggle about the final T +9508.37 Supplier#000000070 FRANCE 17268 Manufacturer#4 INWNH2w,OOWgNDq0BRCcBwOMQc6PdFDc4 16-821-608-1166 ests sleep quickly express ideas. ironic ideas haggle about the final T +9453.01 Supplier#000000802 ROMANIA 10021 Manufacturer#5 ,6HYXb4uaHITmtMBj4Ak57Pd 29-342-882-6463 gular frets. permanently special multipliers believe blithely alongs +9453.01 Supplier#000000802 ROMANIA 13275 Manufacturer#4 ,6HYXb4uaHITmtMBj4Ak57Pd 29-342-882-6463 gular frets. permanently special multipliers believe blithely alongs +9192.10 Supplier#000000115 UNITED KINGDOM 13325 Manufacturer#1 nJ 2t0f7Ve,wL1,6WzGBJLNBUCKlsV 33-597-248-1220 es across the carefully express accounts boost caref +9032.15 Supplier#000000959 GERMANY 4958 Manufacturer#4 8grA EHBnwOZhO 17-108-642-3106 nding dependencies nag furiou +8702.02 Supplier#000000333 RUSSIA 11810 Manufacturer#3 MaVf XgwPdkiX4nfJGOis8Uu2zKiIZH 32-508-202-6136 oss the deposits cajole carefully even pinto beans. regular foxes detect alo +8615.50 Supplier#000000812 FRANCE 10551 Manufacturer#2 8qh4tezyScl5bidLAysvutB,,ZI2dn6xP 16-585-724-6633 y quickly regular deposits? quickly pending packages after the caref +8615.50 Supplier#000000812 FRANCE 13811 Manufacturer#4 8qh4tezyScl5bidLAysvutB,,ZI2dn6xP 16-585-724-6633 y quickly regular deposits? quickly pending packages after the caref +8488.53 Supplier#000000367 RUSSIA 6854 Manufacturer#4 E Sv9brQVf43Mzz 32-458-198-9557 ages. carefully final excuses nag finally. carefully ironic deposits abov +8430.52 Supplier#000000646 FRANCE 11384 Manufacturer#3 IUzsmT,2oBgjhWP2TlXTL6IkJH,4h,1SJRt 16-601-220-5489 ites among the always final ideas kindle according to the theodolites. notornis in +8271.39 Supplier#000000146 RUSSIA 4637 Manufacturer#5 rBDNgCr04x0sfdzD5,gFOutCiG2 32-792-619-3155 s cajole quickly special requests. quickly enticing theodolites h +8096.98 Supplier#000000574 RUSSIA 323 Manufacturer#4 2O8 sy9g2mlBOuEjzj0pA2pevk, 32-866-246-8752 ully after the regular requests. slyly final dependencies wake slyly along the busy deposit +7392.78 Supplier#000000170 UNITED KINGDOM 7655 Manufacturer#2 RtsXQ,SunkA XHy9 33-803-340-5398 ake carefully across the quickly +7205.20 Supplier#000000477 GERMANY 10956 Manufacturer#5 VtaNKN5Mqui5yh7j2ldd5waf 17-180-144-7991 excuses wake express deposits. furiously careful asymptotes according to the carefull +6820.35 Supplier#000000007 UNITED KINGDOM 13217 Manufacturer#5 s,4TicNGB4uO6PaSqNBUq 33-990-965-2201 s unwind silently furiously regular courts. final requests are deposits. requests wake quietly blit +6721.70 Supplier#000000954 FRANCE 4191 Manufacturer#3 P3O5p UFz1QsLmZX 16-537-341-8517 ect blithely blithely final acco +6329.90 Supplier#000000996 GERMANY 10735 Manufacturer#2 Wx4dQwOAwWjfSCGupfrM 17-447-811-3282 ironic forges cajole blithely agai +6173.87 Supplier#000000408 RUSSIA 18139 Manufacturer#1 qcor1u,vJXAokjnL5,dilyYNmh 32-858-724-2950 blithely pending packages cajole furiously slyly pending notornis. slyly final +5364.99 Supplier#000000785 RUSSIA 13784 Manufacturer#4 W VkHBpQyD3qjQjWGpWicOpmILFehmEdWy67kUGY 32-297-653-2203 packages boost carefully. express ideas along +5069.27 Supplier#000000328 GERMANY 16327 Manufacturer#1 SMm24d WG62 17-231-513-5721 he unusual ideas. slyly final packages a +4941.88 Supplier#000000321 ROMANIA 7320 Manufacturer#5 pLngFl5yeMcHyov 29-573-279-1406 y final requests impress s +4672.25 Supplier#000000239 RUSSIA 12238 Manufacturer#1 XO101kgHrJagK2FL1U6QCaTE ncCsMbeuTgK6o8 32-396-654-6826 arls wake furiously deposits. even, regular depen +4586.49 Supplier#000000680 RUSSIA 5679 Manufacturer#3 UhvDfdEfJh,Qbe7VZb8uSGO2TU 0jEa6nXZXE 32-522-382-1620 the regularly regular dependencies. carefully bold excuses under th +4518.31 Supplier#000000149 FRANCE 18344 Manufacturer#5 pVyWsjOidpHKp4NfKU4yLeym 16-660-553-2456 ts detect along the foxes. final Tiresias are. idly pending deposits haggle; even, blithe pin +4315.15 Supplier#000000509 FRANCE 18972 Manufacturer#2 SF7dR8V5pK 16-298-154-3365 ronic orbits are furiously across the requests. quickly express ideas across the special, bold +3526.53 Supplier#000000553 FRANCE 8036 Manufacturer#4 a,liVofXbCJ 16-599-552-3755 lar dinos nag slyly brave +3526.53 Supplier#000000553 FRANCE 17018 Manufacturer#3 a,liVofXbCJ 16-599-552-3755 lar dinos nag slyly brave +3294.68 Supplier#000000350 GERMANY 4841 Manufacturer#4 KIFxV73eovmwhh 17-113-181-4017 e slyly special foxes. furiously unusual deposits detect carefully carefully ruthless foxes. quick +2972.26 Supplier#000000016 RUSSIA 1015 Manufacturer#4 YjP5C55zHDXL7LalK27zfQnwejdpin4AMpvh 32-822-502-4215 ously express ideas haggle quickly dugouts? fu +2963.09 Supplier#000000840 ROMANIA 3080 Manufacturer#2 iYzUIypKhC0Y 29-781-337-5584 eep blithely regular dependencies. blithely regular platelets sublate alongside o +2221.25 Supplier#000000771 ROMANIA 13981 Manufacturer#2 lwZ I15rq9kmZXUNhl 29-986-304-9006 nal foxes eat slyly about the fluffily permanent id +1381.97 Supplier#000000104 FRANCE 18103 Manufacturer#3 Dcl4yGrzqv3OPeRO49bKh78XmQEDR7PBXIs0m 16-434-972-6922 gular ideas. bravely bold deposits haggle through the carefully final deposits. slyly unusual idea +906.07 Supplier#000000138 ROMANIA 8363 Manufacturer#4 utbplAm g7RmxVfYoNdhcrQGWuzRqPe0qHSwbKw 29-533-434-6776 ickly unusual requests cajole. accounts above the furiously special excuses +765.69 Supplier#000000799 RUSSIA 11276 Manufacturer#2 jwFN7ZB3T9sMF 32-579-339-1495 nusual requests. furiously unusual epitaphs integrate. slyly +727.89 Supplier#000000470 ROMANIA 6213 Manufacturer#3 XckbzsAgBLbUkdfjgJEPjmUMTM8ebSMEvI 29-165-289-1523 gular excuses. furiously regular excuses sleep slyly caref +683.07 Supplier#000000651 RUSSIA 4888 Manufacturer#4 oWekiBV6s,1g 32-181-426-4490 ly regular requests cajole abou +167.56 Supplier#000000290 FRANCE 2037 Manufacturer#1 6Bk06GVtwZaKqg01 16-675-286-5102 the theodolites. ironic, ironic deposits above +91.39 Supplier#000000949 UNITED KINGDOM 9430 Manufacturer#2 a,UE,6nRVl2fCphkOoetR1ajIzAEJ1Aa1G1HV 33-332-697-2768 pinto beans. carefully express requests hagg +-314.06 Supplier#000000510 ROMANIA 17242 Manufacturer#4 VmXQl ,vY8JiEseo8Mv4zscvNCfsY 29-207-852-3454 bold deposits. carefully even d +-820.89 Supplier#000000409 GERMANY 2156 Manufacturer#5 LyXUYFz7aXrvy65kKAbTatGzGS,NDBcdtD 17-719-517-9836 y final, slow theodolites. furiously regular req +-845.44 Supplier#000000704 ROMANIA 9926 Manufacturer#5 hQvlBqbqqnA5Dgo1BffRBX78tkkRu 29-300-896-5991 ctions. carefully sly requ +-942.73 Supplier#000000563 GERMANY 5797 Manufacturer#1 Rc7U1cRUhYs03JD 17-108-537-2691 slyly furiously final decoys; silent, special realms poach f -- !select -- -9828.210000000 Supplier#000000647 UNITED KINGDOM 13120 Manufacturer#5 x5U7MBZmwfG9 33-258-202-4782 s the slyly even ideas poach fluffily -9508.370000000 Supplier#000000070 FRANCE 3563 Manufacturer#1 INWNH2w,OOWgNDq0BRCcBwOMQc6PdFDc4 16-821-608-1166 ests sleep quickly express ideas. ironic ideas haggle about the final T -9508.370000000 Supplier#000000070 FRANCE 17268 Manufacturer#4 INWNH2w,OOWgNDq0BRCcBwOMQc6PdFDc4 16-821-608-1166 ests sleep quickly express ideas. ironic ideas haggle about the final T -9453.010000000 Supplier#000000802 ROMANIA 10021 Manufacturer#5 ,6HYXb4uaHITmtMBj4Ak57Pd 29-342-882-6463 gular frets. permanently special multipliers believe blithely alongs -9453.010000000 Supplier#000000802 ROMANIA 13275 Manufacturer#4 ,6HYXb4uaHITmtMBj4Ak57Pd 29-342-882-6463 gular frets. permanently special multipliers believe blithely alongs -9192.100000000 Supplier#000000115 UNITED KINGDOM 13325 Manufacturer#1 nJ 2t0f7Ve,wL1,6WzGBJLNBUCKlsV 33-597-248-1220 es across the carefully express accounts boost caref -9032.150000000 Supplier#000000959 GERMANY 4958 Manufacturer#4 8grA EHBnwOZhO 17-108-642-3106 nding dependencies nag furiou -8702.020000000 Supplier#000000333 RUSSIA 11810 Manufacturer#3 MaVf XgwPdkiX4nfJGOis8Uu2zKiIZH 32-508-202-6136 oss the deposits cajole carefully even pinto beans. regular foxes detect alo -8615.500000000 Supplier#000000812 FRANCE 10551 Manufacturer#2 8qh4tezyScl5bidLAysvutB,,ZI2dn6xP 16-585-724-6633 y quickly regular deposits? quickly pending packages after the caref -8615.500000000 Supplier#000000812 FRANCE 13811 Manufacturer#4 8qh4tezyScl5bidLAysvutB,,ZI2dn6xP 16-585-724-6633 y quickly regular deposits? quickly pending packages after the caref -8488.530000000 Supplier#000000367 RUSSIA 6854 Manufacturer#4 E Sv9brQVf43Mzz 32-458-198-9557 ages. carefully final excuses nag finally. carefully ironic deposits abov -8430.520000000 Supplier#000000646 FRANCE 11384 Manufacturer#3 IUzsmT,2oBgjhWP2TlXTL6IkJH,4h,1SJRt 16-601-220-5489 ites among the always final ideas kindle according to the theodolites. notornis in -8271.390000000 Supplier#000000146 RUSSIA 4637 Manufacturer#5 rBDNgCr04x0sfdzD5,gFOutCiG2 32-792-619-3155 s cajole quickly special requests. quickly enticing theodolites h -8096.980000000 Supplier#000000574 RUSSIA 323 Manufacturer#4 2O8 sy9g2mlBOuEjzj0pA2pevk, 32-866-246-8752 ully after the regular requests. slyly final dependencies wake slyly along the busy deposit -7392.780000000 Supplier#000000170 UNITED KINGDOM 7655 Manufacturer#2 RtsXQ,SunkA XHy9 33-803-340-5398 ake carefully across the quickly -7205.200000000 Supplier#000000477 GERMANY 10956 Manufacturer#5 VtaNKN5Mqui5yh7j2ldd5waf 17-180-144-7991 excuses wake express deposits. furiously careful asymptotes according to the carefull -6820.350000000 Supplier#000000007 UNITED KINGDOM 13217 Manufacturer#5 s,4TicNGB4uO6PaSqNBUq 33-990-965-2201 s unwind silently furiously regular courts. final requests are deposits. requests wake quietly blit -6721.700000000 Supplier#000000954 FRANCE 4191 Manufacturer#3 P3O5p UFz1QsLmZX 16-537-341-8517 ect blithely blithely final acco -6329.900000000 Supplier#000000996 GERMANY 10735 Manufacturer#2 Wx4dQwOAwWjfSCGupfrM 17-447-811-3282 ironic forges cajole blithely agai -6173.870000000 Supplier#000000408 RUSSIA 18139 Manufacturer#1 qcor1u,vJXAokjnL5,dilyYNmh 32-858-724-2950 blithely pending packages cajole furiously slyly pending notornis. slyly final -5364.990000000 Supplier#000000785 RUSSIA 13784 Manufacturer#4 W VkHBpQyD3qjQjWGpWicOpmILFehmEdWy67kUGY 32-297-653-2203 packages boost carefully. express ideas along -5069.270000000 Supplier#000000328 GERMANY 16327 Manufacturer#1 SMm24d WG62 17-231-513-5721 he unusual ideas. slyly final packages a -4941.880000000 Supplier#000000321 ROMANIA 7320 Manufacturer#5 pLngFl5yeMcHyov 29-573-279-1406 y final requests impress s -4672.250000000 Supplier#000000239 RUSSIA 12238 Manufacturer#1 XO101kgHrJagK2FL1U6QCaTE ncCsMbeuTgK6o8 32-396-654-6826 arls wake furiously deposits. even, regular depen -4586.490000000 Supplier#000000680 RUSSIA 5679 Manufacturer#3 UhvDfdEfJh,Qbe7VZb8uSGO2TU 0jEa6nXZXE 32-522-382-1620 the regularly regular dependencies. carefully bold excuses under th -4518.310000000 Supplier#000000149 FRANCE 18344 Manufacturer#5 pVyWsjOidpHKp4NfKU4yLeym 16-660-553-2456 ts detect along the foxes. final Tiresias are. idly pending deposits haggle; even, blithe pin -4315.150000000 Supplier#000000509 FRANCE 18972 Manufacturer#2 SF7dR8V5pK 16-298-154-3365 ronic orbits are furiously across the requests. quickly express ideas across the special, bold -3526.530000000 Supplier#000000553 FRANCE 8036 Manufacturer#4 a,liVofXbCJ 16-599-552-3755 lar dinos nag slyly brave -3526.530000000 Supplier#000000553 FRANCE 17018 Manufacturer#3 a,liVofXbCJ 16-599-552-3755 lar dinos nag slyly brave -3294.680000000 Supplier#000000350 GERMANY 4841 Manufacturer#4 KIFxV73eovmwhh 17-113-181-4017 e slyly special foxes. furiously unusual deposits detect carefully carefully ruthless foxes. quick -2972.260000000 Supplier#000000016 RUSSIA 1015 Manufacturer#4 YjP5C55zHDXL7LalK27zfQnwejdpin4AMpvh 32-822-502-4215 ously express ideas haggle quickly dugouts? fu -2963.090000000 Supplier#000000840 ROMANIA 3080 Manufacturer#2 iYzUIypKhC0Y 29-781-337-5584 eep blithely regular dependencies. blithely regular platelets sublate alongside o -2221.250000000 Supplier#000000771 ROMANIA 13981 Manufacturer#2 lwZ I15rq9kmZXUNhl 29-986-304-9006 nal foxes eat slyly about the fluffily permanent id -1381.970000000 Supplier#000000104 FRANCE 18103 Manufacturer#3 Dcl4yGrzqv3OPeRO49bKh78XmQEDR7PBXIs0m 16-434-972-6922 gular ideas. bravely bold deposits haggle through the carefully final deposits. slyly unusual idea -906.070000000 Supplier#000000138 ROMANIA 8363 Manufacturer#4 utbplAm g7RmxVfYoNdhcrQGWuzRqPe0qHSwbKw 29-533-434-6776 ickly unusual requests cajole. accounts above the furiously special excuses -765.690000000 Supplier#000000799 RUSSIA 11276 Manufacturer#2 jwFN7ZB3T9sMF 32-579-339-1495 nusual requests. furiously unusual epitaphs integrate. slyly -727.890000000 Supplier#000000470 ROMANIA 6213 Manufacturer#3 XckbzsAgBLbUkdfjgJEPjmUMTM8ebSMEvI 29-165-289-1523 gular excuses. furiously regular excuses sleep slyly caref -683.070000000 Supplier#000000651 RUSSIA 4888 Manufacturer#4 oWekiBV6s,1g 32-181-426-4490 ly regular requests cajole abou -167.560000000 Supplier#000000290 FRANCE 2037 Manufacturer#1 6Bk06GVtwZaKqg01 16-675-286-5102 the theodolites. ironic, ironic deposits above -91.390000000 Supplier#000000949 UNITED KINGDOM 9430 Manufacturer#2 a,UE,6nRVl2fCphkOoetR1ajIzAEJ1Aa1G1HV 33-332-697-2768 pinto beans. carefully express requests hagg --314.060000000 Supplier#000000510 ROMANIA 17242 Manufacturer#4 VmXQl ,vY8JiEseo8Mv4zscvNCfsY 29-207-852-3454 bold deposits. carefully even d --820.890000000 Supplier#000000409 GERMANY 2156 Manufacturer#5 LyXUYFz7aXrvy65kKAbTatGzGS,NDBcdtD 17-719-517-9836 y final, slow theodolites. furiously regular req --845.440000000 Supplier#000000704 ROMANIA 9926 Manufacturer#5 hQvlBqbqqnA5Dgo1BffRBX78tkkRu 29-300-896-5991 ctions. carefully sly requ --942.730000000 Supplier#000000563 GERMANY 5797 Manufacturer#1 Rc7U1cRUhYs03JD 17-108-537-2691 slyly furiously final decoys; silent, special realms poach f +9828.21 Supplier#000000647 UNITED KINGDOM 13120 Manufacturer#5 x5U7MBZmwfG9 33-258-202-4782 s the slyly even ideas poach fluffily +9508.37 Supplier#000000070 FRANCE 3563 Manufacturer#1 INWNH2w,OOWgNDq0BRCcBwOMQc6PdFDc4 16-821-608-1166 ests sleep quickly express ideas. ironic ideas haggle about the final T +9508.37 Supplier#000000070 FRANCE 17268 Manufacturer#4 INWNH2w,OOWgNDq0BRCcBwOMQc6PdFDc4 16-821-608-1166 ests sleep quickly express ideas. ironic ideas haggle about the final T +9453.01 Supplier#000000802 ROMANIA 10021 Manufacturer#5 ,6HYXb4uaHITmtMBj4Ak57Pd 29-342-882-6463 gular frets. permanently special multipliers believe blithely alongs +9453.01 Supplier#000000802 ROMANIA 13275 Manufacturer#4 ,6HYXb4uaHITmtMBj4Ak57Pd 29-342-882-6463 gular frets. permanently special multipliers believe blithely alongs +9192.10 Supplier#000000115 UNITED KINGDOM 13325 Manufacturer#1 nJ 2t0f7Ve,wL1,6WzGBJLNBUCKlsV 33-597-248-1220 es across the carefully express accounts boost caref +9032.15 Supplier#000000959 GERMANY 4958 Manufacturer#4 8grA EHBnwOZhO 17-108-642-3106 nding dependencies nag furiou +8702.02 Supplier#000000333 RUSSIA 11810 Manufacturer#3 MaVf XgwPdkiX4nfJGOis8Uu2zKiIZH 32-508-202-6136 oss the deposits cajole carefully even pinto beans. regular foxes detect alo +8615.50 Supplier#000000812 FRANCE 10551 Manufacturer#2 8qh4tezyScl5bidLAysvutB,,ZI2dn6xP 16-585-724-6633 y quickly regular deposits? quickly pending packages after the caref +8615.50 Supplier#000000812 FRANCE 13811 Manufacturer#4 8qh4tezyScl5bidLAysvutB,,ZI2dn6xP 16-585-724-6633 y quickly regular deposits? quickly pending packages after the caref +8488.53 Supplier#000000367 RUSSIA 6854 Manufacturer#4 E Sv9brQVf43Mzz 32-458-198-9557 ages. carefully final excuses nag finally. carefully ironic deposits abov +8430.52 Supplier#000000646 FRANCE 11384 Manufacturer#3 IUzsmT,2oBgjhWP2TlXTL6IkJH,4h,1SJRt 16-601-220-5489 ites among the always final ideas kindle according to the theodolites. notornis in +8271.39 Supplier#000000146 RUSSIA 4637 Manufacturer#5 rBDNgCr04x0sfdzD5,gFOutCiG2 32-792-619-3155 s cajole quickly special requests. quickly enticing theodolites h +8096.98 Supplier#000000574 RUSSIA 323 Manufacturer#4 2O8 sy9g2mlBOuEjzj0pA2pevk, 32-866-246-8752 ully after the regular requests. slyly final dependencies wake slyly along the busy deposit +7392.78 Supplier#000000170 UNITED KINGDOM 7655 Manufacturer#2 RtsXQ,SunkA XHy9 33-803-340-5398 ake carefully across the quickly +7205.20 Supplier#000000477 GERMANY 10956 Manufacturer#5 VtaNKN5Mqui5yh7j2ldd5waf 17-180-144-7991 excuses wake express deposits. furiously careful asymptotes according to the carefull +6820.35 Supplier#000000007 UNITED KINGDOM 13217 Manufacturer#5 s,4TicNGB4uO6PaSqNBUq 33-990-965-2201 s unwind silently furiously regular courts. final requests are deposits. requests wake quietly blit +6721.70 Supplier#000000954 FRANCE 4191 Manufacturer#3 P3O5p UFz1QsLmZX 16-537-341-8517 ect blithely blithely final acco +6329.90 Supplier#000000996 GERMANY 10735 Manufacturer#2 Wx4dQwOAwWjfSCGupfrM 17-447-811-3282 ironic forges cajole blithely agai +6173.87 Supplier#000000408 RUSSIA 18139 Manufacturer#1 qcor1u,vJXAokjnL5,dilyYNmh 32-858-724-2950 blithely pending packages cajole furiously slyly pending notornis. slyly final +5364.99 Supplier#000000785 RUSSIA 13784 Manufacturer#4 W VkHBpQyD3qjQjWGpWicOpmILFehmEdWy67kUGY 32-297-653-2203 packages boost carefully. express ideas along +5069.27 Supplier#000000328 GERMANY 16327 Manufacturer#1 SMm24d WG62 17-231-513-5721 he unusual ideas. slyly final packages a +4941.88 Supplier#000000321 ROMANIA 7320 Manufacturer#5 pLngFl5yeMcHyov 29-573-279-1406 y final requests impress s +4672.25 Supplier#000000239 RUSSIA 12238 Manufacturer#1 XO101kgHrJagK2FL1U6QCaTE ncCsMbeuTgK6o8 32-396-654-6826 arls wake furiously deposits. even, regular depen +4586.49 Supplier#000000680 RUSSIA 5679 Manufacturer#3 UhvDfdEfJh,Qbe7VZb8uSGO2TU 0jEa6nXZXE 32-522-382-1620 the regularly regular dependencies. carefully bold excuses under th +4518.31 Supplier#000000149 FRANCE 18344 Manufacturer#5 pVyWsjOidpHKp4NfKU4yLeym 16-660-553-2456 ts detect along the foxes. final Tiresias are. idly pending deposits haggle; even, blithe pin +4315.15 Supplier#000000509 FRANCE 18972 Manufacturer#2 SF7dR8V5pK 16-298-154-3365 ronic orbits are furiously across the requests. quickly express ideas across the special, bold +3526.53 Supplier#000000553 FRANCE 8036 Manufacturer#4 a,liVofXbCJ 16-599-552-3755 lar dinos nag slyly brave +3526.53 Supplier#000000553 FRANCE 17018 Manufacturer#3 a,liVofXbCJ 16-599-552-3755 lar dinos nag slyly brave +3294.68 Supplier#000000350 GERMANY 4841 Manufacturer#4 KIFxV73eovmwhh 17-113-181-4017 e slyly special foxes. furiously unusual deposits detect carefully carefully ruthless foxes. quick +2972.26 Supplier#000000016 RUSSIA 1015 Manufacturer#4 YjP5C55zHDXL7LalK27zfQnwejdpin4AMpvh 32-822-502-4215 ously express ideas haggle quickly dugouts? fu +2963.09 Supplier#000000840 ROMANIA 3080 Manufacturer#2 iYzUIypKhC0Y 29-781-337-5584 eep blithely regular dependencies. blithely regular platelets sublate alongside o +2221.25 Supplier#000000771 ROMANIA 13981 Manufacturer#2 lwZ I15rq9kmZXUNhl 29-986-304-9006 nal foxes eat slyly about the fluffily permanent id +1381.97 Supplier#000000104 FRANCE 18103 Manufacturer#3 Dcl4yGrzqv3OPeRO49bKh78XmQEDR7PBXIs0m 16-434-972-6922 gular ideas. bravely bold deposits haggle through the carefully final deposits. slyly unusual idea +906.07 Supplier#000000138 ROMANIA 8363 Manufacturer#4 utbplAm g7RmxVfYoNdhcrQGWuzRqPe0qHSwbKw 29-533-434-6776 ickly unusual requests cajole. accounts above the furiously special excuses +765.69 Supplier#000000799 RUSSIA 11276 Manufacturer#2 jwFN7ZB3T9sMF 32-579-339-1495 nusual requests. furiously unusual epitaphs integrate. slyly +727.89 Supplier#000000470 ROMANIA 6213 Manufacturer#3 XckbzsAgBLbUkdfjgJEPjmUMTM8ebSMEvI 29-165-289-1523 gular excuses. furiously regular excuses sleep slyly caref +683.07 Supplier#000000651 RUSSIA 4888 Manufacturer#4 oWekiBV6s,1g 32-181-426-4490 ly regular requests cajole abou +167.56 Supplier#000000290 FRANCE 2037 Manufacturer#1 6Bk06GVtwZaKqg01 16-675-286-5102 the theodolites. ironic, ironic deposits above +91.39 Supplier#000000949 UNITED KINGDOM 9430 Manufacturer#2 a,UE,6nRVl2fCphkOoetR1ajIzAEJ1Aa1G1HV 33-332-697-2768 pinto beans. carefully express requests hagg +-314.06 Supplier#000000510 ROMANIA 17242 Manufacturer#4 VmXQl ,vY8JiEseo8Mv4zscvNCfsY 29-207-852-3454 bold deposits. carefully even d +-820.89 Supplier#000000409 GERMANY 2156 Manufacturer#5 LyXUYFz7aXrvy65kKAbTatGzGS,NDBcdtD 17-719-517-9836 y final, slow theodolites. furiously regular req +-845.44 Supplier#000000704 ROMANIA 9926 Manufacturer#5 hQvlBqbqqnA5Dgo1BffRBX78tkkRu 29-300-896-5991 ctions. carefully sly requ +-942.73 Supplier#000000563 GERMANY 5797 Manufacturer#1 Rc7U1cRUhYs03JD 17-108-537-2691 slyly furiously final decoys; silent, special realms poach f diff --git a/regression-test/data/nereids_tpch_p0/tpch/q22.out b/regression-test/data/nereids_tpch_p0/tpch/q22.out index 8ce4a6cdbc..884b17c0dd 100644 --- a/regression-test/data/nereids_tpch_p0/tpch/q22.out +++ b/regression-test/data/nereids_tpch_p0/tpch/q22.out @@ -1,19 +1,19 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -13 94 714035.050000000 -17 96 722560.150000000 -18 99 738012.520000000 -23 93 708285.250000000 -29 85 632693.460000000 -30 87 646748.020000000 -31 87 647372.500000000 +13 94 714035.05 +17 96 722560.15 +18 99 738012.52 +23 93 708285.25 +29 85 632693.46 +30 87 646748.02 +31 87 647372.50 -- !select -- -13 94 714035.050000000 -17 96 722560.150000000 -18 99 738012.520000000 -23 93 708285.250000000 -29 85 632693.460000000 -30 87 646748.020000000 -31 87 647372.500000000 +13 94 714035.05 +17 96 722560.15 +18 99 738012.52 +23 93 708285.25 +29 85 632693.46 +30 87 646748.02 +31 87 647372.50 diff --git a/regression-test/data/nereids_tpch_p0/tpch/q3.out b/regression-test/data/nereids_tpch_p0/tpch/q3.out index 3986760860..5a217a64cb 100644 --- a/regression-test/data/nereids_tpch_p0/tpch/q3.out +++ b/regression-test/data/nereids_tpch_p0/tpch/q3.out @@ -1,25 +1,25 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -223140 355369.069800000 1995-03-14 0 -584291 354494.731800000 1995-02-21 0 -405063 353125.457700000 1995-03-03 0 -573861 351238.277000000 1995-03-09 0 -554757 349181.742600000 1995-03-14 0 -506021 321075.581000000 1995-03-10 0 -121604 318576.415400000 1995-03-07 0 -108514 314967.075400000 1995-02-20 0 -462502 312604.542000000 1995-03-08 0 -178727 309728.930600000 1995-02-25 0 +223140 355369.0698 1995-03-14 0 +584291 354494.7318 1995-02-21 0 +405063 353125.4577 1995-03-03 0 +573861 351238.2770 1995-03-09 0 +554757 349181.7426 1995-03-14 0 +506021 321075.5810 1995-03-10 0 +121604 318576.4154 1995-03-07 0 +108514 314967.0754 1995-02-20 0 +462502 312604.5420 1995-03-08 0 +178727 309728.9306 1995-02-25 0 -- !select -- -223140 355369.069800000 1995-03-14 0 -584291 354494.731800000 1995-02-21 0 -405063 353125.457700000 1995-03-03 0 -573861 351238.277000000 1995-03-09 0 -554757 349181.742600000 1995-03-14 0 -506021 321075.581000000 1995-03-10 0 -121604 318576.415400000 1995-03-07 0 -108514 314967.075400000 1995-02-20 0 -462502 312604.542000000 1995-03-08 0 -178727 309728.930600000 1995-02-25 0 +223140 355369.0698 1995-03-14 0 +584291 354494.7318 1995-02-21 0 +405063 353125.4577 1995-03-03 0 +573861 351238.2770 1995-03-09 0 +554757 349181.7426 1995-03-14 0 +506021 321075.5810 1995-03-10 0 +121604 318576.4154 1995-03-07 0 +108514 314967.0754 1995-02-20 0 +462502 312604.5420 1995-03-08 0 +178727 309728.9306 1995-02-25 0 diff --git a/regression-test/data/nereids_tpch_p0/tpch/q5.out b/regression-test/data/nereids_tpch_p0/tpch/q5.out index ea6c6dba0d..cbebae337d 100644 --- a/regression-test/data/nereids_tpch_p0/tpch/q5.out +++ b/regression-test/data/nereids_tpch_p0/tpch/q5.out @@ -1,15 +1,15 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -CHINA 7822103.000000000 -INDIA 6376121.508500000 -JAPAN 6000077.218400000 -INDONESIA 5580475.402700000 -VIETNAM 4497840.546600000 +CHINA 7822103.0000 +INDIA 6376121.5085 +JAPAN 6000077.2184 +INDONESIA 5580475.4027 +VIETNAM 4497840.5466 -- !select -- -CHINA 7822103.000000000 -INDIA 6376121.508500000 -JAPAN 6000077.218400000 -INDONESIA 5580475.402700000 -VIETNAM 4497840.546600000 +CHINA 7822103.0000 +INDIA 6376121.5085 +JAPAN 6000077.2184 +INDONESIA 5580475.4027 +VIETNAM 4497840.5466 diff --git a/regression-test/data/nereids_tpch_p0/tpch/q6.out b/regression-test/data/nereids_tpch_p0/tpch/q6.out index 81b7300767..1d56b821ce 100644 --- a/regression-test/data/nereids_tpch_p0/tpch/q6.out +++ b/regression-test/data/nereids_tpch_p0/tpch/q6.out @@ -1,7 +1,7 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -11803420.253400000 +11803420.2534 -- !select -- -11803420.253400000 +11803420.2534 diff --git a/regression-test/data/nereids_tpch_p0/tpch/q7.out b/regression-test/data/nereids_tpch_p0/tpch/q7.out index 84734f1689..19e5872106 100644 --- a/regression-test/data/nereids_tpch_p0/tpch/q7.out +++ b/regression-test/data/nereids_tpch_p0/tpch/q7.out @@ -1,13 +1,13 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -FRANCE GERMANY 1995 4637235.150100000 -FRANCE GERMANY 1996 5224779.573600000 -GERMANY FRANCE 1995 6232818.703700000 -GERMANY FRANCE 1996 5557312.112100000 +FRANCE GERMANY 1995 4637235.1501 +FRANCE GERMANY 1996 5224779.5736 +GERMANY FRANCE 1995 6232818.7037 +GERMANY FRANCE 1996 5557312.1121 -- !select -- -FRANCE GERMANY 1995 4637235.150100000 -FRANCE GERMANY 1996 5224779.573600000 -GERMANY FRANCE 1995 6232818.703700000 -GERMANY FRANCE 1996 5557312.112100000 +FRANCE GERMANY 1995 4637235.1501 +FRANCE GERMANY 1996 5224779.5736 +GERMANY FRANCE 1995 6232818.7037 +GERMANY FRANCE 1996 5557312.1121 diff --git a/regression-test/data/nereids_tpch_p0/tpch/q8.out b/regression-test/data/nereids_tpch_p0/tpch/q8.out index 6f548cc98f..60fa02f361 100644 --- a/regression-test/data/nereids_tpch_p0/tpch/q8.out +++ b/regression-test/data/nereids_tpch_p0/tpch/q8.out @@ -1,9 +1,9 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -1995 0.028648741 -1996 0.018250279 +1995 0.0286 +1996 0.0182 -- !select -- -1995 0.028648741 -1996 0.018250279 +1995 0.0286 +1996 0.0182 diff --git a/regression-test/data/nereids_tpch_p0/tpch/q9.out b/regression-test/data/nereids_tpch_p0/tpch/q9.out index 9d96981953..06659cb47a 100644 --- a/regression-test/data/nereids_tpch_p0/tpch/q9.out +++ b/regression-test/data/nereids_tpch_p0/tpch/q9.out @@ -1,355 +1,355 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -ALGERIA 1998 2321785.368200000 -ALGERIA 1997 3685016.858900000 -ALGERIA 1996 4276597.425300000 -ALGERIA 1995 4418370.415400000 -ALGERIA 1994 3864849.952100000 -ALGERIA 1993 3541051.386500000 -ALGERIA 1992 4310013.348200000 -ARGENTINA 1998 2685983.800500000 -ARGENTINA 1997 4242147.812400000 -ARGENTINA 1996 3907867.010300000 -ARGENTINA 1995 4605921.501100000 -ARGENTINA 1994 3542096.156400000 -ARGENTINA 1993 3949965.938800000 -ARGENTINA 1992 4521180.469500000 -BRAZIL 1998 2778730.393100000 -BRAZIL 1997 4642037.468700000 -BRAZIL 1996 4530304.603400000 -BRAZIL 1995 4502344.865700000 -BRAZIL 1994 4875806.501500000 -BRAZIL 1993 4687478.653100000 -BRAZIL 1992 5035200.046400000 -CANADA 1998 2194509.046500000 -CANADA 1997 3482197.952100000 -CANADA 1996 3712231.281400000 -CANADA 1995 4014814.847600000 -CANADA 1994 4145304.485500000 -CANADA 1993 3787069.604500000 -CANADA 1992 4168009.420100000 -CHINA 1998 3398578.000100000 -CHINA 1997 6358959.333800000 -CHINA 1996 6435158.322900000 -CHINA 1995 6174776.211300000 -CHINA 1994 6385751.081200000 -CHINA 1993 5765034.119400000 -CHINA 1992 6324034.237900000 -EGYPT 1998 2333148.333400000 -EGYPT 1997 3661244.273100000 -EGYPT 1996 3765371.236800000 -EGYPT 1995 4094744.292500000 -EGYPT 1994 3566508.081800000 -EGYPT 1993 3725283.774700000 -EGYPT 1992 3373762.333500000 -ETHIOPIA 1998 1953927.268200000 -ETHIOPIA 1997 3285786.326600000 -ETHIOPIA 1996 3525028.795200000 -ETHIOPIA 1995 3781674.891100000 -ETHIOPIA 1994 3037409.436000000 -ETHIOPIA 1993 3008978.267700000 -ETHIOPIA 1992 2721203.235500000 -FRANCE 1998 2604373.880500000 -FRANCE 1997 3982872.048800000 -FRANCE 1996 3622479.241300000 -FRANCE 1995 4479939.702000000 -FRANCE 1994 3531013.198100000 -FRANCE 1993 4086437.310200000 -FRANCE 1992 3637792.133300000 -GERMANY 1998 3291023.296500000 -GERMANY 1997 5139337.344300000 -GERMANY 1996 4799810.457700000 -GERMANY 1995 5405785.797800000 -GERMANY 1994 4555556.459200000 -GERMANY 1993 4428195.101900000 -GERMANY 1992 4656148.420400000 -INDIA 1998 2591288.187400000 -INDIA 1997 5159562.703300000 -INDIA 1996 5307258.304900000 -INDIA 1995 5148208.790200000 -INDIA 1994 5164001.958200000 -INDIA 1993 4321398.438800000 -INDIA 1992 5297703.693500000 -INDONESIA 1998 3094900.159700000 -INDONESIA 1997 5719773.035800000 -INDONESIA 1996 6037238.599300000 -INDONESIA 1995 5266783.489900000 -INDONESIA 1994 5470762.872900000 -INDONESIA 1993 6189826.661300000 -INDONESIA 1992 4414623.154900000 -IRAN 1998 3214864.120900000 -IRAN 1997 3688049.069100000 -IRAN 1996 3621649.224700000 -IRAN 1995 4420783.420500000 -IRAN 1994 4373984.652300000 -IRAN 1993 3731301.781400000 -IRAN 1992 4417133.366200000 -IRAQ 1998 2338859.409900000 -IRAQ 1997 3622681.564300000 -IRAQ 1996 4762291.872200000 -IRAQ 1995 4558092.735900000 -IRAQ 1994 4951604.169900000 -IRAQ 1993 3830077.991100000 -IRAQ 1992 3938636.487400000 -JAPAN 1998 1849535.080200000 -JAPAN 1997 4068688.853700000 -JAPAN 1996 4044774.759700000 -JAPAN 1995 4793005.802700000 -JAPAN 1994 4114717.056800000 -JAPAN 1993 3614468.748500000 -JAPAN 1992 4266694.470000000 -JORDAN 1998 1811488.071900000 -JORDAN 1997 2951297.867800000 -JORDAN 1996 3302528.306700000 -JORDAN 1995 3221813.999000000 -JORDAN 1994 2417892.092100000 -JORDAN 1993 3107641.766100000 -JORDAN 1992 3316379.058500000 -KENYA 1998 2579075.419000000 -KENYA 1997 2929194.231700000 -KENYA 1996 3569129.561900000 -KENYA 1995 3542889.108700000 -KENYA 1994 3983095.399400000 -KENYA 1993 3713988.970800000 -KENYA 1992 3304641.834000000 -MOROCCO 1998 1815334.818000000 -MOROCCO 1997 3693214.844700000 -MOROCCO 1996 4116175.923000000 -MOROCCO 1995 3515127.140200000 -MOROCCO 1994 4003072.112000000 -MOROCCO 1993 3599199.667900000 -MOROCCO 1992 3958335.422400000 -MOZAMBIQUE 1998 1620428.734600000 -MOZAMBIQUE 1997 2802166.647300000 -MOZAMBIQUE 1996 2409955.175500000 -MOZAMBIQUE 1995 2771602.627400000 -MOZAMBIQUE 1994 2548226.215800000 -MOZAMBIQUE 1993 2843748.905300000 -MOZAMBIQUE 1992 2556501.094300000 -PERU 1998 2036430.360200000 -PERU 1997 4064142.409100000 -PERU 1996 4068678.567100000 -PERU 1995 4657694.841200000 -PERU 1994 4731959.465500000 -PERU 1993 4144006.661000000 -PERU 1992 3754635.007800000 -ROMANIA 1998 1992773.681100000 -ROMANIA 1997 2854639.868000000 -ROMANIA 1996 3139337.302900000 -ROMANIA 1995 3222153.377600000 -ROMANIA 1994 3222844.319000000 -ROMANIA 1993 3488994.028800000 -ROMANIA 1992 3029274.442000000 -RUSSIA 1998 2339865.663500000 -RUSSIA 1997 4153619.542400000 -RUSSIA 1996 3772067.404100000 -RUSSIA 1995 4704988.860700000 -RUSSIA 1994 4479082.869400000 -RUSSIA 1993 4767719.979100000 -RUSSIA 1992 4533465.559000000 -SAUDI ARABIA 1998 3386948.956400000 -SAUDI ARABIA 1997 5425980.337300000 -SAUDI ARABIA 1996 5227607.167700000 -SAUDI ARABIA 1995 4506731.641100000 -SAUDI ARABIA 1994 4698658.742500000 -SAUDI ARABIA 1993 5493626.528500000 -SAUDI ARABIA 1992 4573560.015000000 -UNITED KINGDOM 1998 2252021.513700000 -UNITED KINGDOM 1997 4343926.802600000 -UNITED KINGDOM 1996 4189476.306500000 -UNITED KINGDOM 1995 4469569.882900000 -UNITED KINGDOM 1994 4410094.626400000 -UNITED KINGDOM 1993 4054677.105000000 -UNITED KINGDOM 1992 3978688.883100000 -UNITED STATES 1998 2238771.558100000 -UNITED STATES 1997 4135581.573400000 -UNITED STATES 1996 3624013.266000000 -UNITED STATES 1995 3892244.517200000 -UNITED STATES 1994 3289224.113800000 -UNITED STATES 1993 3626170.202800000 -UNITED STATES 1992 3993973.499700000 -VIETNAM 1998 1924313.486200000 -VIETNAM 1997 3436195.370900000 -VIETNAM 1996 4017288.892700000 -VIETNAM 1995 3644054.137200000 -VIETNAM 1994 4141277.666500000 -VIETNAM 1993 2556114.169300000 -VIETNAM 1992 4090524.490500000 +ALGERIA 1998 2321785.3682 +ALGERIA 1997 3685016.8589 +ALGERIA 1996 4276597.4253 +ALGERIA 1995 4418370.4154 +ALGERIA 1994 3864849.9521 +ALGERIA 1993 3541051.3865 +ALGERIA 1992 4310013.3482 +ARGENTINA 1998 2685983.8005 +ARGENTINA 1997 4242147.8124 +ARGENTINA 1996 3907867.0103 +ARGENTINA 1995 4605921.5011 +ARGENTINA 1994 3542096.1564 +ARGENTINA 1993 3949965.9388 +ARGENTINA 1992 4521180.4695 +BRAZIL 1998 2778730.3931 +BRAZIL 1997 4642037.4687 +BRAZIL 1996 4530304.6034 +BRAZIL 1995 4502344.8657 +BRAZIL 1994 4875806.5015 +BRAZIL 1993 4687478.6531 +BRAZIL 1992 5035200.0464 +CANADA 1998 2194509.0465 +CANADA 1997 3482197.9521 +CANADA 1996 3712231.2814 +CANADA 1995 4014814.8476 +CANADA 1994 4145304.4855 +CANADA 1993 3787069.6045 +CANADA 1992 4168009.4201 +CHINA 1998 3398578.0001 +CHINA 1997 6358959.3338 +CHINA 1996 6435158.3229 +CHINA 1995 6174776.2113 +CHINA 1994 6385751.0812 +CHINA 1993 5765034.1194 +CHINA 1992 6324034.2379 +EGYPT 1998 2333148.3334 +EGYPT 1997 3661244.2731 +EGYPT 1996 3765371.2368 +EGYPT 1995 4094744.2925 +EGYPT 1994 3566508.0818 +EGYPT 1993 3725283.7747 +EGYPT 1992 3373762.3335 +ETHIOPIA 1998 1953927.2682 +ETHIOPIA 1997 3285786.3266 +ETHIOPIA 1996 3525028.7952 +ETHIOPIA 1995 3781674.8911 +ETHIOPIA 1994 3037409.4360 +ETHIOPIA 1993 3008978.2677 +ETHIOPIA 1992 2721203.2355 +FRANCE 1998 2604373.8805 +FRANCE 1997 3982872.0488 +FRANCE 1996 3622479.2413 +FRANCE 1995 4479939.7020 +FRANCE 1994 3531013.1981 +FRANCE 1993 4086437.3102 +FRANCE 1992 3637792.1333 +GERMANY 1998 3291023.2965 +GERMANY 1997 5139337.3443 +GERMANY 1996 4799810.4577 +GERMANY 1995 5405785.7978 +GERMANY 1994 4555556.4592 +GERMANY 1993 4428195.1019 +GERMANY 1992 4656148.4204 +INDIA 1998 2591288.1874 +INDIA 1997 5159562.7033 +INDIA 1996 5307258.3049 +INDIA 1995 5148208.7902 +INDIA 1994 5164001.9582 +INDIA 1993 4321398.4388 +INDIA 1992 5297703.6935 +INDONESIA 1998 3094900.1597 +INDONESIA 1997 5719773.0358 +INDONESIA 1996 6037238.5993 +INDONESIA 1995 5266783.4899 +INDONESIA 1994 5470762.8729 +INDONESIA 1993 6189826.6613 +INDONESIA 1992 4414623.1549 +IRAN 1998 3214864.1209 +IRAN 1997 3688049.0691 +IRAN 1996 3621649.2247 +IRAN 1995 4420783.4205 +IRAN 1994 4373984.6523 +IRAN 1993 3731301.7814 +IRAN 1992 4417133.3662 +IRAQ 1998 2338859.4099 +IRAQ 1997 3622681.5643 +IRAQ 1996 4762291.8722 +IRAQ 1995 4558092.7359 +IRAQ 1994 4951604.1699 +IRAQ 1993 3830077.9911 +IRAQ 1992 3938636.4874 +JAPAN 1998 1849535.0802 +JAPAN 1997 4068688.8537 +JAPAN 1996 4044774.7597 +JAPAN 1995 4793005.8027 +JAPAN 1994 4114717.0568 +JAPAN 1993 3614468.7485 +JAPAN 1992 4266694.4700 +JORDAN 1998 1811488.0719 +JORDAN 1997 2951297.8678 +JORDAN 1996 3302528.3067 +JORDAN 1995 3221813.9990 +JORDAN 1994 2417892.0921 +JORDAN 1993 3107641.7661 +JORDAN 1992 3316379.0585 +KENYA 1998 2579075.4190 +KENYA 1997 2929194.2317 +KENYA 1996 3569129.5619 +KENYA 1995 3542889.1087 +KENYA 1994 3983095.3994 +KENYA 1993 3713988.9708 +KENYA 1992 3304641.8340 +MOROCCO 1998 1815334.8180 +MOROCCO 1997 3693214.8447 +MOROCCO 1996 4116175.9230 +MOROCCO 1995 3515127.1402 +MOROCCO 1994 4003072.1120 +MOROCCO 1993 3599199.6679 +MOROCCO 1992 3958335.4224 +MOZAMBIQUE 1998 1620428.7346 +MOZAMBIQUE 1997 2802166.6473 +MOZAMBIQUE 1996 2409955.1755 +MOZAMBIQUE 1995 2771602.6274 +MOZAMBIQUE 1994 2548226.2158 +MOZAMBIQUE 1993 2843748.9053 +MOZAMBIQUE 1992 2556501.0943 +PERU 1998 2036430.3602 +PERU 1997 4064142.4091 +PERU 1996 4068678.5671 +PERU 1995 4657694.8412 +PERU 1994 4731959.4655 +PERU 1993 4144006.6610 +PERU 1992 3754635.0078 +ROMANIA 1998 1992773.6811 +ROMANIA 1997 2854639.8680 +ROMANIA 1996 3139337.3029 +ROMANIA 1995 3222153.3776 +ROMANIA 1994 3222844.3190 +ROMANIA 1993 3488994.0288 +ROMANIA 1992 3029274.4420 +RUSSIA 1998 2339865.6635 +RUSSIA 1997 4153619.5424 +RUSSIA 1996 3772067.4041 +RUSSIA 1995 4704988.8607 +RUSSIA 1994 4479082.8694 +RUSSIA 1993 4767719.9791 +RUSSIA 1992 4533465.5590 +SAUDI ARABIA 1998 3386948.9564 +SAUDI ARABIA 1997 5425980.3373 +SAUDI ARABIA 1996 5227607.1677 +SAUDI ARABIA 1995 4506731.6411 +SAUDI ARABIA 1994 4698658.7425 +SAUDI ARABIA 1993 5493626.5285 +SAUDI ARABIA 1992 4573560.0150 +UNITED KINGDOM 1998 2252021.5137 +UNITED KINGDOM 1997 4343926.8026 +UNITED KINGDOM 1996 4189476.3065 +UNITED KINGDOM 1995 4469569.8829 +UNITED KINGDOM 1994 4410094.6264 +UNITED KINGDOM 1993 4054677.1050 +UNITED KINGDOM 1992 3978688.8831 +UNITED STATES 1998 2238771.5581 +UNITED STATES 1997 4135581.5734 +UNITED STATES 1996 3624013.2660 +UNITED STATES 1995 3892244.5172 +UNITED STATES 1994 3289224.1138 +UNITED STATES 1993 3626170.2028 +UNITED STATES 1992 3993973.4997 +VIETNAM 1998 1924313.4862 +VIETNAM 1997 3436195.3709 +VIETNAM 1996 4017288.8927 +VIETNAM 1995 3644054.1372 +VIETNAM 1994 4141277.6665 +VIETNAM 1993 2556114.1693 +VIETNAM 1992 4090524.4905 -- !select -- -ALGERIA 1998 2321785.368200000 -ALGERIA 1997 3685016.858900000 -ALGERIA 1996 4276597.425300000 -ALGERIA 1995 4418370.415400000 -ALGERIA 1994 3864849.952100000 -ALGERIA 1993 3541051.386500000 -ALGERIA 1992 4310013.348200000 -ARGENTINA 1998 2685983.800500000 -ARGENTINA 1997 4242147.812400000 -ARGENTINA 1996 3907867.010300000 -ARGENTINA 1995 4605921.501100000 -ARGENTINA 1994 3542096.156400000 -ARGENTINA 1993 3949965.938800000 -ARGENTINA 1992 4521180.469500000 -BRAZIL 1998 2778730.393100000 -BRAZIL 1997 4642037.468700000 -BRAZIL 1996 4530304.603400000 -BRAZIL 1995 4502344.865700000 -BRAZIL 1994 4875806.501500000 -BRAZIL 1993 4687478.653100000 -BRAZIL 1992 5035200.046400000 -CANADA 1998 2194509.046500000 -CANADA 1997 3482197.952100000 -CANADA 1996 3712231.281400000 -CANADA 1995 4014814.847600000 -CANADA 1994 4145304.485500000 -CANADA 1993 3787069.604500000 -CANADA 1992 4168009.420100000 -CHINA 1998 3398578.000100000 -CHINA 1997 6358959.333800000 -CHINA 1996 6435158.322900000 -CHINA 1995 6174776.211300000 -CHINA 1994 6385751.081200000 -CHINA 1993 5765034.119400000 -CHINA 1992 6324034.237900000 -EGYPT 1998 2333148.333400000 -EGYPT 1997 3661244.273100000 -EGYPT 1996 3765371.236800000 -EGYPT 1995 4094744.292500000 -EGYPT 1994 3566508.081800000 -EGYPT 1993 3725283.774700000 -EGYPT 1992 3373762.333500000 -ETHIOPIA 1998 1953927.268200000 -ETHIOPIA 1997 3285786.326600000 -ETHIOPIA 1996 3525028.795200000 -ETHIOPIA 1995 3781674.891100000 -ETHIOPIA 1994 3037409.436000000 -ETHIOPIA 1993 3008978.267700000 -ETHIOPIA 1992 2721203.235500000 -FRANCE 1998 2604373.880500000 -FRANCE 1997 3982872.048800000 -FRANCE 1996 3622479.241300000 -FRANCE 1995 4479939.702000000 -FRANCE 1994 3531013.198100000 -FRANCE 1993 4086437.310200000 -FRANCE 1992 3637792.133300000 -GERMANY 1998 3291023.296500000 -GERMANY 1997 5139337.344300000 -GERMANY 1996 4799810.457700000 -GERMANY 1995 5405785.797800000 -GERMANY 1994 4555556.459200000 -GERMANY 1993 4428195.101900000 -GERMANY 1992 4656148.420400000 -INDIA 1998 2591288.187400000 -INDIA 1997 5159562.703300000 -INDIA 1996 5307258.304900000 -INDIA 1995 5148208.790200000 -INDIA 1994 5164001.958200000 -INDIA 1993 4321398.438800000 -INDIA 1992 5297703.693500000 -INDONESIA 1998 3094900.159700000 -INDONESIA 1997 5719773.035800000 -INDONESIA 1996 6037238.599300000 -INDONESIA 1995 5266783.489900000 -INDONESIA 1994 5470762.872900000 -INDONESIA 1993 6189826.661300000 -INDONESIA 1992 4414623.154900000 -IRAN 1998 3214864.120900000 -IRAN 1997 3688049.069100000 -IRAN 1996 3621649.224700000 -IRAN 1995 4420783.420500000 -IRAN 1994 4373984.652300000 -IRAN 1993 3731301.781400000 -IRAN 1992 4417133.366200000 -IRAQ 1998 2338859.409900000 -IRAQ 1997 3622681.564300000 -IRAQ 1996 4762291.872200000 -IRAQ 1995 4558092.735900000 -IRAQ 1994 4951604.169900000 -IRAQ 1993 3830077.991100000 -IRAQ 1992 3938636.487400000 -JAPAN 1998 1849535.080200000 -JAPAN 1997 4068688.853700000 -JAPAN 1996 4044774.759700000 -JAPAN 1995 4793005.802700000 -JAPAN 1994 4114717.056800000 -JAPAN 1993 3614468.748500000 -JAPAN 1992 4266694.470000000 -JORDAN 1998 1811488.071900000 -JORDAN 1997 2951297.867800000 -JORDAN 1996 3302528.306700000 -JORDAN 1995 3221813.999000000 -JORDAN 1994 2417892.092100000 -JORDAN 1993 3107641.766100000 -JORDAN 1992 3316379.058500000 -KENYA 1998 2579075.419000000 -KENYA 1997 2929194.231700000 -KENYA 1996 3569129.561900000 -KENYA 1995 3542889.108700000 -KENYA 1994 3983095.399400000 -KENYA 1993 3713988.970800000 -KENYA 1992 3304641.834000000 -MOROCCO 1998 1815334.818000000 -MOROCCO 1997 3693214.844700000 -MOROCCO 1996 4116175.923000000 -MOROCCO 1995 3515127.140200000 -MOROCCO 1994 4003072.112000000 -MOROCCO 1993 3599199.667900000 -MOROCCO 1992 3958335.422400000 -MOZAMBIQUE 1998 1620428.734600000 -MOZAMBIQUE 1997 2802166.647300000 -MOZAMBIQUE 1996 2409955.175500000 -MOZAMBIQUE 1995 2771602.627400000 -MOZAMBIQUE 1994 2548226.215800000 -MOZAMBIQUE 1993 2843748.905300000 -MOZAMBIQUE 1992 2556501.094300000 -PERU 1998 2036430.360200000 -PERU 1997 4064142.409100000 -PERU 1996 4068678.567100000 -PERU 1995 4657694.841200000 -PERU 1994 4731959.465500000 -PERU 1993 4144006.661000000 -PERU 1992 3754635.007800000 -ROMANIA 1998 1992773.681100000 -ROMANIA 1997 2854639.868000000 -ROMANIA 1996 3139337.302900000 -ROMANIA 1995 3222153.377600000 -ROMANIA 1994 3222844.319000000 -ROMANIA 1993 3488994.028800000 -ROMANIA 1992 3029274.442000000 -RUSSIA 1998 2339865.663500000 -RUSSIA 1997 4153619.542400000 -RUSSIA 1996 3772067.404100000 -RUSSIA 1995 4704988.860700000 -RUSSIA 1994 4479082.869400000 -RUSSIA 1993 4767719.979100000 -RUSSIA 1992 4533465.559000000 -SAUDI ARABIA 1998 3386948.956400000 -SAUDI ARABIA 1997 5425980.337300000 -SAUDI ARABIA 1996 5227607.167700000 -SAUDI ARABIA 1995 4506731.641100000 -SAUDI ARABIA 1994 4698658.742500000 -SAUDI ARABIA 1993 5493626.528500000 -SAUDI ARABIA 1992 4573560.015000000 -UNITED KINGDOM 1998 2252021.513700000 -UNITED KINGDOM 1997 4343926.802600000 -UNITED KINGDOM 1996 4189476.306500000 -UNITED KINGDOM 1995 4469569.882900000 -UNITED KINGDOM 1994 4410094.626400000 -UNITED KINGDOM 1993 4054677.105000000 -UNITED KINGDOM 1992 3978688.883100000 -UNITED STATES 1998 2238771.558100000 -UNITED STATES 1997 4135581.573400000 -UNITED STATES 1996 3624013.266000000 -UNITED STATES 1995 3892244.517200000 -UNITED STATES 1994 3289224.113800000 -UNITED STATES 1993 3626170.202800000 -UNITED STATES 1992 3993973.499700000 -VIETNAM 1998 1924313.486200000 -VIETNAM 1997 3436195.370900000 -VIETNAM 1996 4017288.892700000 -VIETNAM 1995 3644054.137200000 -VIETNAM 1994 4141277.666500000 -VIETNAM 1993 2556114.169300000 -VIETNAM 1992 4090524.490500000 +ALGERIA 1998 2321785.3682 +ALGERIA 1997 3685016.8589 +ALGERIA 1996 4276597.4253 +ALGERIA 1995 4418370.4154 +ALGERIA 1994 3864849.9521 +ALGERIA 1993 3541051.3865 +ALGERIA 1992 4310013.3482 +ARGENTINA 1998 2685983.8005 +ARGENTINA 1997 4242147.8124 +ARGENTINA 1996 3907867.0103 +ARGENTINA 1995 4605921.5011 +ARGENTINA 1994 3542096.1564 +ARGENTINA 1993 3949965.9388 +ARGENTINA 1992 4521180.4695 +BRAZIL 1998 2778730.3931 +BRAZIL 1997 4642037.4687 +BRAZIL 1996 4530304.6034 +BRAZIL 1995 4502344.8657 +BRAZIL 1994 4875806.5015 +BRAZIL 1993 4687478.6531 +BRAZIL 1992 5035200.0464 +CANADA 1998 2194509.0465 +CANADA 1997 3482197.9521 +CANADA 1996 3712231.2814 +CANADA 1995 4014814.8476 +CANADA 1994 4145304.4855 +CANADA 1993 3787069.6045 +CANADA 1992 4168009.4201 +CHINA 1998 3398578.0001 +CHINA 1997 6358959.3338 +CHINA 1996 6435158.3229 +CHINA 1995 6174776.2113 +CHINA 1994 6385751.0812 +CHINA 1993 5765034.1194 +CHINA 1992 6324034.2379 +EGYPT 1998 2333148.3334 +EGYPT 1997 3661244.2731 +EGYPT 1996 3765371.2368 +EGYPT 1995 4094744.2925 +EGYPT 1994 3566508.0818 +EGYPT 1993 3725283.7747 +EGYPT 1992 3373762.3335 +ETHIOPIA 1998 1953927.2682 +ETHIOPIA 1997 3285786.3266 +ETHIOPIA 1996 3525028.7952 +ETHIOPIA 1995 3781674.8911 +ETHIOPIA 1994 3037409.4360 +ETHIOPIA 1993 3008978.2677 +ETHIOPIA 1992 2721203.2355 +FRANCE 1998 2604373.8805 +FRANCE 1997 3982872.0488 +FRANCE 1996 3622479.2413 +FRANCE 1995 4479939.7020 +FRANCE 1994 3531013.1981 +FRANCE 1993 4086437.3102 +FRANCE 1992 3637792.1333 +GERMANY 1998 3291023.2965 +GERMANY 1997 5139337.3443 +GERMANY 1996 4799810.4577 +GERMANY 1995 5405785.7978 +GERMANY 1994 4555556.4592 +GERMANY 1993 4428195.1019 +GERMANY 1992 4656148.4204 +INDIA 1998 2591288.1874 +INDIA 1997 5159562.7033 +INDIA 1996 5307258.3049 +INDIA 1995 5148208.7902 +INDIA 1994 5164001.9582 +INDIA 1993 4321398.4388 +INDIA 1992 5297703.6935 +INDONESIA 1998 3094900.1597 +INDONESIA 1997 5719773.0358 +INDONESIA 1996 6037238.5993 +INDONESIA 1995 5266783.4899 +INDONESIA 1994 5470762.8729 +INDONESIA 1993 6189826.6613 +INDONESIA 1992 4414623.1549 +IRAN 1998 3214864.1209 +IRAN 1997 3688049.0691 +IRAN 1996 3621649.2247 +IRAN 1995 4420783.4205 +IRAN 1994 4373984.6523 +IRAN 1993 3731301.7814 +IRAN 1992 4417133.3662 +IRAQ 1998 2338859.4099 +IRAQ 1997 3622681.5643 +IRAQ 1996 4762291.8722 +IRAQ 1995 4558092.7359 +IRAQ 1994 4951604.1699 +IRAQ 1993 3830077.9911 +IRAQ 1992 3938636.4874 +JAPAN 1998 1849535.0802 +JAPAN 1997 4068688.8537 +JAPAN 1996 4044774.7597 +JAPAN 1995 4793005.8027 +JAPAN 1994 4114717.0568 +JAPAN 1993 3614468.7485 +JAPAN 1992 4266694.4700 +JORDAN 1998 1811488.0719 +JORDAN 1997 2951297.8678 +JORDAN 1996 3302528.3067 +JORDAN 1995 3221813.9990 +JORDAN 1994 2417892.0921 +JORDAN 1993 3107641.7661 +JORDAN 1992 3316379.0585 +KENYA 1998 2579075.4190 +KENYA 1997 2929194.2317 +KENYA 1996 3569129.5619 +KENYA 1995 3542889.1087 +KENYA 1994 3983095.3994 +KENYA 1993 3713988.9708 +KENYA 1992 3304641.8340 +MOROCCO 1998 1815334.8180 +MOROCCO 1997 3693214.8447 +MOROCCO 1996 4116175.9230 +MOROCCO 1995 3515127.1402 +MOROCCO 1994 4003072.1120 +MOROCCO 1993 3599199.6679 +MOROCCO 1992 3958335.4224 +MOZAMBIQUE 1998 1620428.7346 +MOZAMBIQUE 1997 2802166.6473 +MOZAMBIQUE 1996 2409955.1755 +MOZAMBIQUE 1995 2771602.6274 +MOZAMBIQUE 1994 2548226.2158 +MOZAMBIQUE 1993 2843748.9053 +MOZAMBIQUE 1992 2556501.0943 +PERU 1998 2036430.3602 +PERU 1997 4064142.4091 +PERU 1996 4068678.5671 +PERU 1995 4657694.8412 +PERU 1994 4731959.4655 +PERU 1993 4144006.6610 +PERU 1992 3754635.0078 +ROMANIA 1998 1992773.6811 +ROMANIA 1997 2854639.8680 +ROMANIA 1996 3139337.3029 +ROMANIA 1995 3222153.3776 +ROMANIA 1994 3222844.3190 +ROMANIA 1993 3488994.0288 +ROMANIA 1992 3029274.4420 +RUSSIA 1998 2339865.6635 +RUSSIA 1997 4153619.5424 +RUSSIA 1996 3772067.4041 +RUSSIA 1995 4704988.8607 +RUSSIA 1994 4479082.8694 +RUSSIA 1993 4767719.9791 +RUSSIA 1992 4533465.5590 +SAUDI ARABIA 1998 3386948.9564 +SAUDI ARABIA 1997 5425980.3373 +SAUDI ARABIA 1996 5227607.1677 +SAUDI ARABIA 1995 4506731.6411 +SAUDI ARABIA 1994 4698658.7425 +SAUDI ARABIA 1993 5493626.5285 +SAUDI ARABIA 1992 4573560.0150 +UNITED KINGDOM 1998 2252021.5137 +UNITED KINGDOM 1997 4343926.8026 +UNITED KINGDOM 1996 4189476.3065 +UNITED KINGDOM 1995 4469569.8829 +UNITED KINGDOM 1994 4410094.6264 +UNITED KINGDOM 1993 4054677.1050 +UNITED KINGDOM 1992 3978688.8831 +UNITED STATES 1998 2238771.5581 +UNITED STATES 1997 4135581.5734 +UNITED STATES 1996 3624013.2660 +UNITED STATES 1995 3892244.5172 +UNITED STATES 1994 3289224.1138 +UNITED STATES 1993 3626170.2028 +UNITED STATES 1992 3993973.4997 +VIETNAM 1998 1924313.4862 +VIETNAM 1997 3436195.3709 +VIETNAM 1996 4017288.8927 +VIETNAM 1995 3644054.1372 +VIETNAM 1994 4141277.6665 +VIETNAM 1993 2556114.1693 +VIETNAM 1992 4090524.4905 diff --git a/regression-test/suites/nereids_tpch_p0/ddl/customer.sql b/regression-test/suites/nereids_tpch_p0/ddl/customer.sql index 00b1b98d48..3ea59421b6 100644 --- a/regression-test/suites/nereids_tpch_p0/ddl/customer.sql +++ b/regression-test/suites/nereids_tpch_p0/ddl/customer.sql @@ -4,7 +4,7 @@ CREATE TABLE IF NOT EXISTS customer ( C_ADDRESS VARCHAR(40) NOT NULL, C_NATIONKEY INTEGER NOT NULL, C_PHONE CHAR(15) NOT NULL, - C_ACCTBAL DECIMAL(15,2) NOT NULL, + C_ACCTBAL DECIMALV3(15,2) NOT NULL, C_MKTSEGMENT CHAR(10) NOT NULL, C_COMMENT VARCHAR(117) NOT NULL ) diff --git a/regression-test/suites/nereids_tpch_p0/ddl/lineitem.sql b/regression-test/suites/nereids_tpch_p0/ddl/lineitem.sql index 2b4cb77143..4576307071 100644 --- a/regression-test/suites/nereids_tpch_p0/ddl/lineitem.sql +++ b/regression-test/suites/nereids_tpch_p0/ddl/lineitem.sql @@ -3,10 +3,10 @@ CREATE TABLE IF NOT EXISTS lineitem ( L_PARTKEY INTEGER NOT NULL, L_SUPPKEY INTEGER NOT NULL, L_LINENUMBER INTEGER NOT NULL, - L_QUANTITY DECIMAL(15,2) NOT NULL, - L_EXTENDEDPRICE DECIMAL(15,2) NOT NULL, - L_DISCOUNT DECIMAL(15,2) NOT NULL, - L_TAX DECIMAL(15,2) NOT NULL, + L_QUANTITY DECIMALV3(15,2) NOT NULL, + L_EXTENDEDPRICE DECIMALV3(15,2) NOT NULL, + L_DISCOUNT DECIMALV3(15,2) NOT NULL, + L_TAX DECIMALV3(15,2) NOT NULL, L_RETURNFLAG CHAR(1) NOT NULL, L_LINESTATUS CHAR(1) NOT NULL, L_SHIPDATE DATE NOT NULL, diff --git a/regression-test/suites/nereids_tpch_p0/ddl/orders.sql b/regression-test/suites/nereids_tpch_p0/ddl/orders.sql index caeaa34150..2257d3781f 100644 --- a/regression-test/suites/nereids_tpch_p0/ddl/orders.sql +++ b/regression-test/suites/nereids_tpch_p0/ddl/orders.sql @@ -2,7 +2,7 @@ CREATE TABLE IF NOT EXISTS orders ( O_ORDERKEY INTEGER NOT NULL, O_CUSTKEY INTEGER NOT NULL, O_ORDERSTATUS CHAR(1) NOT NULL, - O_TOTALPRICE DECIMAL(15,2) NOT NULL, + O_TOTALPRICE DECIMALV3(15,2) NOT NULL, O_ORDERDATE DATE NOT NULL, O_ORDERPRIORITY CHAR(15) NOT NULL, O_CLERK CHAR(15) NOT NULL, diff --git a/regression-test/suites/nereids_tpch_p0/ddl/part.sql b/regression-test/suites/nereids_tpch_p0/ddl/part.sql index 994b6e66d5..c22c1c9680 100644 --- a/regression-test/suites/nereids_tpch_p0/ddl/part.sql +++ b/regression-test/suites/nereids_tpch_p0/ddl/part.sql @@ -6,7 +6,7 @@ CREATE TABLE IF NOT EXISTS part ( P_TYPE VARCHAR(25) NOT NULL, P_SIZE INTEGER NOT NULL, P_CONTAINER CHAR(10) NOT NULL, - P_RETAILPRICE DECIMAL(15,2) NOT NULL, + P_RETAILPRICE DECIMALV3(15,2) NOT NULL, P_COMMENT VARCHAR(23) NOT NULL ) DUPLICATE KEY(P_PARTKEY, P_NAME) diff --git a/regression-test/suites/nereids_tpch_p0/ddl/partsupp.sql b/regression-test/suites/nereids_tpch_p0/ddl/partsupp.sql index be186a29db..c6d6dc6c8d 100644 --- a/regression-test/suites/nereids_tpch_p0/ddl/partsupp.sql +++ b/regression-test/suites/nereids_tpch_p0/ddl/partsupp.sql @@ -2,7 +2,7 @@ CREATE TABLE IF NOT EXISTS partsupp ( PS_PARTKEY INTEGER NOT NULL, PS_SUPPKEY INTEGER NOT NULL, PS_AVAILQTY INTEGER NOT NULL, - PS_SUPPLYCOST DECIMAL(15,2) NOT NULL, + PS_SUPPLYCOST DECIMALV3(15,2) NOT NULL, PS_COMMENT VARCHAR(199) NOT NULL ) DUPLICATE KEY(PS_PARTKEY, PS_SUPPKEY) diff --git a/regression-test/suites/nereids_tpch_p0/ddl/supplier.sql b/regression-test/suites/nereids_tpch_p0/ddl/supplier.sql index 7214eaebd1..1f633ac3d3 100644 --- a/regression-test/suites/nereids_tpch_p0/ddl/supplier.sql +++ b/regression-test/suites/nereids_tpch_p0/ddl/supplier.sql @@ -4,7 +4,7 @@ CREATE TABLE IF NOT EXISTS supplier ( S_ADDRESS VARCHAR(40) NOT NULL, S_NATIONKEY INTEGER NOT NULL, S_PHONE CHAR(15) NOT NULL, - S_ACCTBAL DECIMAL(15,2) NOT NULL, + S_ACCTBAL DECIMALV3(15,2) NOT NULL, S_COMMENT VARCHAR(101) NOT NULL ) DUPLICATE KEY(S_SUPPKEY, S_NAME)