[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
This commit is contained in:
@ -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())))))
|
||||
* <p>
|
||||
* refer paper: WinMagic - Subquery Elimination Using Window Aggregation
|
||||
* <p>
|
||||
* TODO: use materialized view pattern match to do outer and inner tree match.
|
||||
*/
|
||||
|
||||
public class AggScalarSubQueryToWindowFunction extends DefaultPlanRewriter<JobContext> implements CustomRewriter {
|
||||
private static final Set<Class<? extends AggregateFunction>> SUPPORTED_FUNCTION = ImmutableSet.of(
|
||||
Min.class, Max.class, Count.class, Sum.class, Avg.class
|
||||
);
|
||||
private static final Set<Class<? extends LogicalPlan>> LEFT_SUPPORTED_PLAN = ImmutableSet.of(
|
||||
LogicalRelation.class, LogicalJoin.class, LogicalProject.class, LogicalFilter.class, LogicalLimit.class
|
||||
);
|
||||
private static final Set<Class<? extends LogicalPlan>> RIGHT_SUPPORTED_PLAN = ImmutableSet.of(
|
||||
LogicalRelation.class, LogicalJoin.class, LogicalProject.class, LogicalFilter.class, LogicalAggregate.class
|
||||
);
|
||||
private List<LogicalPlan> outerPlans = null;
|
||||
private List<LogicalPlan> innerPlans = null;
|
||||
private LogicalAggregate aggOp = null;
|
||||
private List<AggregateFunction> functions = null;
|
||||
|
||||
private static final Set<Class<? extends LogicalPlan>> OUTER_SUPPORTED_PLAN = ImmutableSet.of(
|
||||
LogicalJoin.class,
|
||||
LogicalProject.class,
|
||||
LogicalRelation.class
|
||||
);
|
||||
|
||||
private static final Set<Class<? extends LogicalPlan>> INNER_SUPPORTED_PLAN = ImmutableSet.of(
|
||||
LogicalAggregate.class,
|
||||
LogicalFilter.class,
|
||||
LogicalJoin.class,
|
||||
LogicalProject.class,
|
||||
LogicalRelation.class
|
||||
);
|
||||
|
||||
private final List<LogicalPlan> outerPlans = Lists.newArrayList();
|
||||
private final List<LogicalPlan> innerPlans = Lists.newArrayList();
|
||||
private final List<AggregateFunction> functions = Lists.newArrayList();
|
||||
private final Map<Expression, Expression> 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<? extends Plan> filter, JobContext context) {
|
||||
LogicalApply<Plan, LogicalAggregate<Plan>> 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<Plan, LogicalAggregate<Plan>> checkPattern(LogicalFilter<? extends Plan> 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<LogicalApply<Plan, Plan>> findApply(LogicalFilter<? extends Plan> filter) {
|
||||
return Optional.of(filter.child())
|
||||
.map(p -> p instanceof LogicalProject ? p.child(0) : p)
|
||||
.filter(LogicalApply.class::isInstance)
|
||||
.map(p -> (LogicalApply<Plan, Plan>) p);
|
||||
}
|
||||
|
||||
private boolean check(LogicalFilter<? extends Plan> filter, LogicalApply<Plan, LogicalAggregate<Plan>> apply) {
|
||||
LogicalPlan outer = ((LogicalPlan) apply.child(0));
|
||||
LogicalPlan inner = ((LogicalPlan) apply.child(1));
|
||||
outerPlans = PlanCollector.INSTANCE.collect(outer);
|
||||
innerPlans = PlanCollector.INSTANCE.collect(inner);
|
||||
Optional<LogicalFilter> innerFilter = innerPlans.stream()
|
||||
.filter(LogicalFilter.class::isInstance)
|
||||
.map(LogicalFilter.class::cast).findFirst();
|
||||
return innerFilter.isPresent()
|
||||
&& checkPlanType() && checkAggType()
|
||||
private boolean check(LogicalFilter<? extends Plan> outerFilter, LogicalApply<Plan, Plan> 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<Plan, Plan> 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<LogicalAggregate> 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<LogicalAggregate<Plan>> aggSet = innerPlans.stream().filter(LogicalAggregate.class::isInstance)
|
||||
.map(p -> (LogicalAggregate<Plan>) 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<AggregateFunction>) ExpressionUtils.<AggregateFunction>collectAll(
|
||||
LogicalAggregate<Plan> 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<? extends Plan> outerFilter) {
|
||||
List<LogicalFilter<Plan>> innerFilters = innerPlans.stream()
|
||||
.filter(LogicalFilter.class::isInstance)
|
||||
.map(p -> (LogicalFilter<Plan>) p).collect(Collectors.toList());
|
||||
if (innerFilters.size() != 1) {
|
||||
return false;
|
||||
}
|
||||
Set<Expression> outerConjunctSet = Sets.newHashSet(outerFilter.getConjuncts());
|
||||
Set<Expression> innerConjunctSet = innerFilters.get(0).getConjuncts().stream()
|
||||
.map(e -> ExpressionUtils.replace(e, innerOuterSlotMap))
|
||||
.collect(Collectors.toSet());
|
||||
Iterator<Expression> innerIterator = innerConjunctSet.iterator();
|
||||
// inner predicate should be the sub-set of outer predicate.
|
||||
while (innerIterator.hasNext()) {
|
||||
Expression innerExpr = innerIterator.next();
|
||||
Iterator<Expression> 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<Plan, Plan>) p)
|
||||
.noneMatch(j -> j.getOnClauseCondition().isPresent())
|
||||
&& innerPlans.stream()
|
||||
.filter(LogicalJoin.class::isInstance)
|
||||
.map(p -> (LogicalJoin<Plan, Plan>) 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<Plan>) p)
|
||||
.allMatch(p -> p.getExpressions().stream().allMatch(SlotReference.class::isInstance))
|
||||
&& innerPlans.stream()
|
||||
.filter(LogicalProject.class::isInstance)
|
||||
.map(p -> (LogicalProject<Plan>) 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<Expression> correlatedSlots) {
|
||||
List<LogicalRelation> outerTables = outerPlans.stream().filter(LogicalRelation.class::isInstance)
|
||||
.map(LogicalRelation.class::cast)
|
||||
@ -176,15 +254,21 @@ public class AggScalarSubQueryToWindowFunction extends DefaultPlanRewriter<JobCo
|
||||
.map(LogicalRelation.class::cast)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
Set<Long> outerIds = outerTables.stream().map(node -> node.getTable().getId()).collect(Collectors.toSet());
|
||||
Set<Long> innerIds = innerTables.stream().map(node -> node.getTable().getId()).collect(Collectors.toSet());
|
||||
|
||||
Set<Long> outerCopy = Sets.newHashSet(outerIds);
|
||||
outerIds.removeAll(innerIds);
|
||||
innerIds.removeAll(outerCopy);
|
||||
if (outerIds.isEmpty() || !innerIds.isEmpty()) {
|
||||
List<Long> outerIds = outerTables.stream().map(node -> node.getTable().getId()).collect(Collectors.toList());
|
||||
List<Long> 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<ExprId> correlatedRelationOutput = outerTables.stream()
|
||||
.filter(node -> outerIds.contains(node.getTable().getId()))
|
||||
@ -194,26 +278,28 @@ public class AggScalarSubQueryToWindowFunction extends DefaultPlanRewriter<JobCo
|
||||
.allMatch(e -> correlatedRelationOutput.contains(e.getExprId()));
|
||||
}
|
||||
|
||||
private boolean checkPredicate(Set<Expression> outerConjuncts, Set<Expression> innerConjuncts) {
|
||||
Iterator<Expression> innerIter = innerConjuncts.iterator();
|
||||
// inner predicate should be the sub-set of outer predicate.
|
||||
while (innerIter.hasNext()) {
|
||||
Expression innerExpr = innerIter.next();
|
||||
Iterator<Expression> outerIter = outerConjuncts.iterator();
|
||||
while (outerIter.hasNext()) {
|
||||
Expression outerExpr = outerIter.next();
|
||||
if (ExpressionIdenticalChecker.INSTANCE.check(innerExpr, outerExpr)) {
|
||||
innerIter.remove();
|
||||
outerIter.remove();
|
||||
private void createSlotMapping(List<LogicalRelation> outerTables, List<LogicalRelation> 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<? extends Plan> filter, LogicalApply<Plan, LogicalAggregate<Plan>> apply) {
|
||||
LogicalAggregate<Plan> agg = apply.right();
|
||||
private Plan rewrite(LogicalFilter<? extends Plan> filter, LogicalApply<Plan, Plan> apply) {
|
||||
Preconditions.checkArgument(apply.right() instanceof LogicalAggregate,
|
||||
"right child of Apply should be LogicalAggregate");
|
||||
LogicalAggregate<Plan> agg = (LogicalAggregate<Plan>) 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<JobCo
|
||||
// cast(l_quantity#id1 as decimal(27, 9)) < `0.2 * avg(l_quantity)`#id2
|
||||
// and
|
||||
// 0.2 * avg(l_quantity#id3) as `0.2 * l_quantity`#id2
|
||||
// is agg's output expression
|
||||
// is aggregate's output expression
|
||||
// we change it to
|
||||
// cast(l_quantity#id1 as decimal(27, 9)) < 0.2 * `avg(l_quantity#id1) over(window)`#id4
|
||||
// and
|
||||
@ -241,9 +327,6 @@ public class AggScalarSubQueryToWindowFunction extends DefaultPlanRewriter<JobCo
|
||||
windowFilterConjunct = PlanUtils.maybeCommuteComparisonPredicate(
|
||||
(ComparisonPredicate) windowFilterConjunct, apply.left());
|
||||
|
||||
// build window function, replace the slot
|
||||
List<Expression> 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<JobCo
|
||||
}
|
||||
|
||||
WindowExpression windowFunction = createWindowFunction(apply.getCorrelationSlot(),
|
||||
function.withChildren(windowAggSlots));
|
||||
(AggregateFunction) ExpressionUtils.replace(function, innerOuterSlotMap));
|
||||
NamedExpression windowFunctionAlias = new Alias(windowFunction, windowFunction.toSql());
|
||||
|
||||
// build filter conjunct, get the alias of the agg output and extract its child.
|
||||
@ -260,8 +343,8 @@ public class AggScalarSubQueryToWindowFunction extends DefaultPlanRewriter<JobCo
|
||||
NamedExpression aggOut = agg.getOutputExpressions().get(0);
|
||||
Expression aggOutExpr = aggOut.child(0);
|
||||
// change the agg function to window function alias.
|
||||
aggOutExpr = MapReplacer.INSTANCE.replace(aggOutExpr, ImmutableMap
|
||||
.of(AggregateFunction.class, e -> 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<JobCo
|
||||
windowFilterConjunct = windowFilterConjunct
|
||||
.withChildren(windowFilterConjunct.child(0), aggOutExpr);
|
||||
|
||||
LogicalFilter newFilter = ((LogicalFilter) filter.withChildren(apply.left()));
|
||||
LogicalWindow newWindow = new LogicalWindow<>(ImmutableList.of(windowFunctionAlias), newFilter);
|
||||
LogicalFilter windowFilter = new LogicalFilter<>(ImmutableSet.of(windowFilterConjunct), newWindow);
|
||||
LogicalFilter<Plan> newFilter = (LogicalFilter<Plan>) filter.withChildren(apply.left());
|
||||
LogicalWindow<Plan> newWindow = new LogicalWindow<>(ImmutableList.of(windowFunctionAlias), newFilter);
|
||||
LogicalFilter<Plan> windowFilter = new LogicalFilter<>(ImmutableSet.of(windowFilterConjunct), newWindow);
|
||||
return windowFilter;
|
||||
}
|
||||
|
||||
@ -281,24 +364,6 @@ public class AggScalarSubQueryToWindowFunction extends DefaultPlanRewriter<JobCo
|
||||
return new WindowExpression(function, correlatedSlots, Collections.emptyList());
|
||||
}
|
||||
|
||||
private static class PlanCollector extends DefaultPlanVisitor<Void, List<LogicalPlan>> {
|
||||
public static final PlanCollector INSTANCE = new PlanCollector();
|
||||
|
||||
public List<LogicalPlan> collect(LogicalPlan plan) {
|
||||
List<LogicalPlan> buffer = Lists.newArrayList();
|
||||
plan.accept(this, buffer);
|
||||
return buffer;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Void visit(Plan plan, List<LogicalPlan> 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<Boolean, Expression> {
|
||||
public static final ExpressionIdenticalChecker INSTANCE = new ExpressionIdenticalChecker();
|
||||
|
||||
@ -322,64 +387,24 @@ public class AggScalarSubQueryToWindowFunction extends DefaultPlanRewriter<JobCo
|
||||
return true;
|
||||
}
|
||||
|
||||
private boolean isSameObjects(Object... o) {
|
||||
Preconditions.checkArgument(o.length % 2 == 0);
|
||||
for (int i = 0; i < o.length; i += 2) {
|
||||
if (!Objects.equals(o[i], o[i + 1])) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
private boolean isSameOperator(Expression expression, Expression expression1, Object... o) {
|
||||
return isSameObjects(o) && isSameChild(expression, expression1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Boolean visit(Expression expression, Expression expression1) {
|
||||
return isClassMatch(expression, expression1) && isSameChild(expression, expression1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Boolean visitNamedExpression(NamedExpression namedExpression, Expression expr) {
|
||||
return isClassMatch(namedExpression, expr)
|
||||
&& isSameOperator(namedExpression, expr, namedExpression.getName(),
|
||||
((NamedExpression) expr).getName());
|
||||
public Boolean visitSlotReference(SlotReference slotReference, Expression other) {
|
||||
return slotReference.equals(other);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Boolean visitLiteral(Literal literal, Expression expr) {
|
||||
return isClassMatch(literal, expr)
|
||||
&& isSameOperator(literal, expr, literal.getValue(), ((Literal) expr).getValue());
|
||||
public Boolean visitLiteral(Literal literal, Expression other) {
|
||||
return literal.equals(other);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Boolean visitEqualTo(EqualTo equalTo, Expression expr) {
|
||||
return isSameChild(equalTo, expr) || isSameChild(equalTo.commute(), expr);
|
||||
}
|
||||
}
|
||||
|
||||
private static class MapReplacer extends DefaultExpressionRewriter<Map<Class<? extends Expression>,
|
||||
Function<Expression, Expression>>> {
|
||||
public static final MapReplacer INSTANCE = new MapReplacer();
|
||||
|
||||
public Expression replace(Expression e, Map<Class<? extends Expression>,
|
||||
Function<Expression, Expression>> context) {
|
||||
return e.accept(this, context);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Expression visit(Expression e, Map<Class<? extends Expression>,
|
||||
Function<Expression, Expression>> 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -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<FunctionSignature> SIGNATURES = ImmutableList.of(
|
||||
FunctionSignature.ret(DoubleType.INSTANCE).args(TinyIntType.INSTANCE),
|
||||
|
||||
@ -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<FunctionSignature> SIGNATURES = ImmutableList.of(
|
||||
// count(*)
|
||||
|
||||
@ -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);
|
||||
}
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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<FunctionSignature> SIGNATURES = ImmutableList.of(
|
||||
FunctionSignature.ret(BigIntType.INSTANCE).args(TinyIntType.INSTANCE),
|
||||
|
||||
@ -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 {
|
||||
}
|
||||
@ -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);
|
||||
|
||||
@ -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) {
|
||||
|
||||
@ -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
|
||||
|
||||
|
||||
@ -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
|
||||
|
||||
|
||||
File diff suppressed because it is too large
Load Diff
@ -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
|
||||
|
||||
|
||||
@ -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
|
||||
|
||||
|
||||
@ -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
|
||||
|
||||
|
||||
@ -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
|
||||
|
||||
|
||||
@ -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
|
||||
|
||||
|
||||
@ -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
|
||||
|
||||
|
||||
@ -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
|
||||
|
||||
|
||||
@ -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
|
||||
|
||||
|
||||
@ -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
|
||||
|
||||
|
||||
@ -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
|
||||
|
||||
|
||||
@ -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
|
||||
|
||||
|
||||
@ -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
|
||||
|
||||
|
||||
@ -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
|
||||
|
||||
|
||||
@ -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
|
||||
)
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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)
|
||||
|
||||
Reference in New Issue
Block a user