From 53660b4482e2a53b5f9aee5a2f2d1e9067f255bd Mon Sep 17 00:00:00 2001 From: Michael Date: Thu, 14 Apr 2016 21:18:37 -0700 Subject: [PATCH] TestHistogram zipfan dataset as well the interfaces of the sort based joins The parallel sort related elements, mainly includes: 1. parallel sort running frameworks including the histograms, histogrammerge, forward operators. 2. two types of histogram and their inner algorithms covering streaming based numeric histogram and ternary based string histogram. 3. provides fours samplers, bernoulli, reservoir, random and chain for further optimizations. 4. Some testSet and testCodes. Further works: Split the types of implementations separately into two sub-branches to accommodate the basic types of parallel sort. 1. Running framework as well as the numeric type. 2. Extend the framework onto the string case. Make the inner structures according to the normal rules of hyracks. Change-Id: I8eb7f0dddcd4b754b1cbe273ef8db5be966654d5 --- .../AbstractSortMergeJoinPOperator.java | 81 + .../physical/BandSortMergeJoinPOperator.java | 82 + .../asterix/optimizer/base/AnalysisUtil.java | 146 +- .../SetAsterixPhysicalOperatorsRule.java | 40 + .../algebra/base/PhysicalOperatorTag.java | 1 + .../SortMergeJoinExpressionAnnotation.java | 64 + .../api/context/IHyracksTaskContext.java | 5 + .../value/ITuplePartitionComputerFactory.java | 4 +- .../org/apache/hyracks/control/nc/Task.java | 12 + .../FieldHashPartitionComputerFactory.java | 3 +- .../RandomPartitionComputerFactory.java | 4 +- .../partition/RepartitionComputerFactory.java | 5 +- .../FieldRangePartitionComputerFactory.java | 3 +- ...doopHashTuplePartitionComputerFactory.java | 4 +- ...titionerTuplePartitionComputerFactory.java | 4 +- ...titionerTuplePartitionComputerFactory.java | 4 +- ...artitioningShuffleConnectorDescriptor.java | 6 +- ...reMToNPartitioningConnectorDescriptor.java | 4 +- .../MToNPartitioningConnectorDescriptor.java | 10 +- ...artitioningMergingConnectorDescriptor.java | 2 +- ...tioningWithMessageConnectorDescriptor.java | 2 +- .../GraceHashJoinOperatorNodePushable.java | 4 +- ...oinPartitionBuildOperatorNodePushable.java | 5 +- .../HybridHashJoinOperatorDescriptor.java | 13 +- .../InMemoryHashJoinOperatorDescriptor.java | 4 +- .../std/parallel/HistogramAlgorithm.java | 30 + .../dataflow/std/parallel/IDTHistogram.java | 30 + .../dataflow/std/parallel/IHistogram.java | 56 + .../std/parallel/IIterativeHistogram.java | 39 + .../std/parallel/ISequentialAccessor.java | 42 + .../std/parallel/ISequentialTrie.java | 33 + .../std/parallel/IStatisticEntity.java | 33 + .../dataflow/std/parallel/IStatisticNode.java | 32 + .../std/parallel/IStatisticTernaryNode.java | 63 + .../parallel/base/AbstractSamplingWriter.java | 228 + .../parallel/base/DefaultSamplingWriter.java | 76 + ...eldRangePartitionDelayComputerFactory.java | 117 + ...togramConnectorPolicyAssignmentPolicy.java | 43 + .../base/MaterializingSampleTaskState.java | 90 + .../base/MergeOrderedSampleWriter.java | 177 + .../parallel/base/OrderedSamplingWriter.java | 180 + .../base/ParallelRangeMapTaskState.java | 173 + .../parallel/base/QuantileSamplingWriter.java | 99 + .../AbstractSampleOperatorDescriptor.java | 202 + .../BlockingForwardOperatorDescriptor.java | 127 + .../ForwardOperatorNodePushable.java | 67 + ...aterializingForwardOperatorDescriptor.java | 167 + ...MaterializingSampleOperatorDescriptor.java | 69 + ...terializingSampleOperatorNodePushable.java | 151 + .../MergeSampleOperatorDescriptor.java | 150 + .../MergeSampleOperatorNodePushable.java | 151 + .../SampleReaderOperatorNodePushable.java | 107 + .../AbstractStreamingHistogram.java | 226 + .../structures/DTStreamingHistogram.java | 1018 ++ .../structures/TernaryIterativeHistogram.java | 407 + .../histogram/terneray/MemoryTernaryNode.java | 160 + .../terneray/SequentialAccessor.java | 99 + .../histogram/terneray/TernaryMemoryTrie.java | 338 + .../parallel/sampler/BernoulliSampler.java | 94 + .../std/parallel/sampler/ChainSampler.java | 107 + .../std/parallel/sampler/ISampler.java | 46 + .../std/parallel/sampler/RandomSampler.java | 102 + .../parallel/sampler/ReservoirSampler.java | 202 + .../std/parallel/util/DualSerialEntry.java | 74 + .../std/parallel/util/HistogramUtils.java | 203 + .../std/parallel/util/ValueSerialEntry.java | 56 + .../dataflow/std/parallel/TrieTests.java | 115 + .../std/parallel/TypeTranslateTest.java | 84 + .../data/skew/zipfan.tbl | 10000 ++++++++++++++++ .../ConnectorPolicyAssignmentPolicy.java | 42 + .../tests/integration/SampleBandJoinTest.java | 119 + .../tests/integration/SampleForwardTest.java | 664 + .../tests/integration/SamplerTest.java | 118 + .../tests/util/SampleAlgorithmTests.java | 898 ++ .../hyracks/test/support/TestTaskContext.java | 10 + 75 files changed, 18390 insertions(+), 36 deletions(-) create mode 100644 asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/AbstractSortMergeJoinPOperator.java create mode 100644 asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BandSortMergeJoinPOperator.java create mode 100644 hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/SortMergeJoinExpressionAnnotation.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/HistogramAlgorithm.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/IDTHistogram.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/IHistogram.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/IIterativeHistogram.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/ISequentialAccessor.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/ISequentialTrie.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/IStatisticEntity.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/IStatisticNode.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/IStatisticTernaryNode.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/AbstractSamplingWriter.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/DefaultSamplingWriter.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/FieldRangePartitionDelayComputerFactory.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/HistogramConnectorPolicyAssignmentPolicy.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/MaterializingSampleTaskState.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/MergeOrderedSampleWriter.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/OrderedSamplingWriter.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/ParallelRangeMapTaskState.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/QuantileSamplingWriter.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/AbstractSampleOperatorDescriptor.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/BlockingForwardOperatorDescriptor.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/ForwardOperatorNodePushable.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/MaterializingForwardOperatorDescriptor.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/MaterializingSampleOperatorDescriptor.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/MaterializingSampleOperatorNodePushable.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/MergeSampleOperatorDescriptor.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/MergeSampleOperatorNodePushable.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/SampleReaderOperatorNodePushable.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/structures/AbstractStreamingHistogram.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/structures/DTStreamingHistogram.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/structures/TernaryIterativeHistogram.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/terneray/MemoryTernaryNode.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/terneray/SequentialAccessor.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/terneray/TernaryMemoryTrie.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/sampler/BernoulliSampler.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/sampler/ChainSampler.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/sampler/ISampler.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/sampler/RandomSampler.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/sampler/ReservoirSampler.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/util/DualSerialEntry.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/util/HistogramUtils.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/util/ValueSerialEntry.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/parallel/TrieTests.java create mode 100644 hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/parallel/TypeTranslateTest.java create mode 100644 hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/data/skew/zipfan.tbl create mode 100644 hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/ConnectorPolicyAssignmentPolicy.java create mode 100644 hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/SampleBandJoinTest.java create mode 100644 hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/SampleForwardTest.java create mode 100644 hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/SamplerTest.java create mode 100644 hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/util/SampleAlgorithmTests.java diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/AbstractSortMergeJoinPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/AbstractSortMergeJoinPOperator.java new file mode 100644 index 00000000000..5d8cb07c138 --- /dev/null +++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/AbstractSortMergeJoinPOperator.java @@ -0,0 +1,81 @@ +/* + * 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.asterix.algebra.operators.physical; + +import java.util.List; + +import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException; +import org.apache.hyracks.algebricks.common.utils.Pair; +import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression; +import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator; +import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext; +import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable; +import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator.JoinKind; +import org.apache.hyracks.algebricks.core.algebra.operators.physical.AbstractJoinPOperator; +import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector; +import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirements; + +public abstract class AbstractSortMergeJoinPOperator extends AbstractJoinPOperator { + // Current for Single column Band, Theta join, will be extended onto multiple columns as well as Metric and Skyline join. + + protected List keysLeftTopPartition; + protected List keysRightTopPartition; + protected Pair limitRange; + protected ILogicalExpression partitionGranularity; + + public AbstractSortMergeJoinPOperator(JoinKind kind, JoinPartitioningType partitioningType, + List sideLeft, List sideRight, + Pair range, ILogicalExpression gran) { + super(kind, partitioningType); + this.keysLeftTopPartition = sideLeft; + this.keysRightTopPartition = sideRight; + this.limitRange = range; + this.partitionGranularity = gran; + } + + public List getKeysLeftTopPartition() { + return keysLeftTopPartition; + } + + public List getKeysRightTopPartition() { + return keysRightTopPartition; + } + + public Pair getLimitRange() { + return limitRange; + } + + public ILogicalExpression getGranularity() { + return partitionGranularity; + } + + @Override + public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op, + IPhysicalPropertiesVector reqdByParent, IOptimizationContext context) { + // TODO Auto-generated method stub + return null; + } + + @Override + public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) + throws AlgebricksException { + // TODO Auto-generated method stub + + } +} diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BandSortMergeJoinPOperator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BandSortMergeJoinPOperator.java new file mode 100644 index 00000000000..3b3a9af94e3 --- /dev/null +++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/operators/physical/BandSortMergeJoinPOperator.java @@ -0,0 +1,82 @@ +/* + * 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.asterix.algebra.operators.physical; + +import java.util.List; +import java.util.logging.Logger; + +import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException; +import org.apache.hyracks.algebricks.common.utils.Pair; +import org.apache.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder; +import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression; +import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator; +import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable; +import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag; +import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator.JoinKind; +import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema; +import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext; + +public class BandSortMergeJoinPOperator extends AbstractSortMergeJoinPOperator { + private final int memSizeInFrames; + private final int maxInputBuildSizeInFrames; + private final int slidingWindowSizeInFrames; + private final int aveRecordsPerFrame; + private final double fudgeFactor; + + private static final Logger LOGGER = Logger.getLogger(BandSortMergeJoinPOperator.class.getName()); + + public BandSortMergeJoinPOperator(JoinKind kind, JoinPartitioningType partitioningType, + List sideLeft, List sideRight, + Pair range, ILogicalExpression gran, int memSizeInFrames, + int maxInputBuildSizeInFrames, int slidingWindowSizeInFrames, int aveRecordsPerFrame, double fudgeFactor) { + super(kind, partitioningType, sideLeft, sideRight, range, gran); + this.memSizeInFrames = memSizeInFrames; + this.maxInputBuildSizeInFrames = maxInputBuildSizeInFrames; + this.slidingWindowSizeInFrames = slidingWindowSizeInFrames; + this.aveRecordsPerFrame = aveRecordsPerFrame; + this.fudgeFactor = fudgeFactor; + + LOGGER.fine("BandSortMergeJoinPOperator constructed with: JoinKind: " + kind + " JoinPartitioningType=" + + partitioningType + " List=" + sideLeft + " List=" + range + " ILogicalExpression=" + gran + + " memSizeInFrames=" + memSizeInFrames + " maxInputBuildSizeInFrames=" + maxInputBuildSizeInFrames + + " slidingWindowSizeInFrames=" + slidingWindowSizeInFrames + " aveRecordsPerFrame=" + + aveRecordsPerFrame + " fudgeFactor=" + fudgeFactor); + } + + @Override + public PhysicalOperatorTag getOperatorTag() { + // TODO Auto-generated method stub + return PhysicalOperatorTag.BAND_SORTMERGE_JOIN; + } + + @Override + public boolean isMicroOperator() { + // TODO Auto-generated method stub + return false; + } + + @Override + public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op, + IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema) + throws AlgebricksException { + // TODO Auto-generated method stub + + } +} diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AnalysisUtil.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AnalysisUtil.java index beefc4695b7..e38076c0b08 100644 --- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AnalysisUtil.java +++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/AnalysisUtil.java @@ -19,8 +19,10 @@ package org.apache.asterix.optimizer.base; import java.util.ArrayList; +import java.util.Collection; import java.util.List; +import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractDataSourceOperator; import org.apache.asterix.metadata.declared.AqlSourceId; import org.apache.asterix.om.functions.AsterixBuiltinFunctions; import org.apache.asterix.optimizer.rules.am.AccessMethodUtils; @@ -31,12 +33,20 @@ import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator; import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag; import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag; +import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable; import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression; +import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression.FunctionKind; +import org.apache.hyracks.algebricks.core.algebra.expressions.ScalarFunctionCallExpression; +import org.apache.hyracks.algebricks.core.algebra.expressions.SortMergeJoinExpressionAnnotation.SortMergeJoinType; +import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression; import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression; +import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions; +import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions.ComparisonKind; import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier; -import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractDataSourceOperator; +import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator; import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator; import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator; +import org.apache.hyracks.algebricks.core.config.AlgebricksConfig; public class AnalysisUtil { /* @@ -84,6 +94,140 @@ public static boolean isRunnableFieldAccessFunction(FunctionIdentifier fid) { return fieldAccessFunctions.contains(fid); } + private static SortMergeJoinType getSortMergeJoinable(ILogicalExpression e, Collection inLeftAll, + Collection inRightAll, List outLeftFields, + List outRightFields, List> outBandRanges) { + AbstractFunctionCallExpression fexp = (AbstractFunctionCallExpression) e; + FunctionIdentifier fi = fexp.getFunctionIdentifier(); + if (fi.equals(AlgebricksBuiltinFunctions.AND)) { + SortMergeJoinType retType = SortMergeJoinType.NESTLOOP; + for (Mutable a : fexp.getArguments()) { + SortMergeJoinType childType = getSortMergeJoinable(a.getValue(), inLeftAll, inRightAll, outLeftFields, + outRightFields, outBandRanges); + if (SortMergeJoinType.BAND == childType) + retType = SortMergeJoinType.BAND; + else if (retType != SortMergeJoinType.BAND && SortMergeJoinType.THETA == childType) + retType = SortMergeJoinType.THETA; + // else if ... + // For Metric and Skyline join type in the future. + } + return retType; + } else { + ComparisonKind ck = AlgebricksBuiltinFunctions.getComparisonType(fi); + if (null == ck || ck == ComparisonKind.EQ) { + AlgebricksConfig.ALGEBRICKS_LOGGER + .info("// SortMerge joinable encounter equal or fj condition -- Condition for" + e + ": " + ck); + return null; + } + ILogicalExpression opLeft = fexp.getArguments().get(0).getValue(); + ILogicalExpression opRight = fexp.getArguments().get(1).getValue(); + if (opLeft.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL + && opRight.getExpressionTag() == LogicalExpressionTag.CONSTANT) { + ScalarFunctionCallExpression sfe = (ScalarFunctionCallExpression) opLeft; + if (FunctionKind.SCALAR != sfe.getKind() + && AsterixBuiltinFunctions.NUMERIC_SUBTRACT != sfe.getFunctionIdentifier()) + return null; + for (int j = 0; j < 2; j++) { + LogicalVariable varLeft = ((VariableReferenceExpression) (sfe.getArguments().get(j).getValue())) + .getVariableReference(); + LogicalVariable varRight = ((VariableReferenceExpression) (sfe.getArguments().get((j + 1) % 2) + .getValue())).getVariableReference(); + // We did not provide the merge of the partial ConstantExpression. + if (inLeftAll.contains(varLeft) && inRightAll.contains(varRight)) { + for (int i = 0; i < outLeftFields.size(); i++) { + if (varLeft.equals(outLeftFields.get(i)) && varRight.equals(outRightFields.get(i))) { + return updateAndGetRanges(outLeftFields, outRightFields, outBandRanges, i, ck, opRight); + } + } + outLeftFields.add(varLeft); + outRightFields.add(varRight); + outBandRanges.add(new Pair(null, null)); + return updateAndGetRanges(outLeftFields, outRightFields, outBandRanges, + outBandRanges.size() - 1, ck, opRight); + } + } + } + } + return SortMergeJoinType.NESTLOOP; + } + + private static SortMergeJoinType updateAndGetRanges(List outLeftFields, + List outRightFields, List> bandRanges, + int index, ComparisonKind ck, ILogicalExpression value) { + switch (ck) { + case GT: + case GE: { + // Add the ConstantExpression merge here in future. + if (bandRanges.size() < index || null == bandRanges.get(index)) { + AlgebricksConfig.ALGEBRICKS_LOGGER.info("// Band condition left insert exception -- Condition for" + + value + ": " + ck); + } + bandRanges.get(index).first = value; + break; + } + case LT: + case LE: { + if (bandRanges.size() < index || null == bandRanges.get(index)) { + AlgebricksConfig.ALGEBRICKS_LOGGER.info("// Band condition right insert exception -- Condition for" + + value + ": " + ck); + } + bandRanges.get(index).second = value; + break; + } + default: + break; + } + + if (isBandRange(outLeftFields, outRightFields, bandRanges)) + return SortMergeJoinType.BAND; + else if (isThetaRange(outLeftFields, outRightFields, bandRanges)) + return SortMergeJoinType.THETA; + // Further for Metric and Skyline join. + else + return SortMergeJoinType.NESTLOOP; + } + + private static boolean isBandRange(List leftVars, List rightVars, + List> bandRanges) { + if (leftVars.size() != rightVars.size() || leftVars.size() != bandRanges.size()) + return false; + for (int i = 0; i < bandRanges.size(); i++) { + if (bandRanges.get(i).first != null && bandRanges.get(i).second != null) + return true; + } + return false; + } + + private static boolean isThetaRange(List leftVars, List rightVars, + List> bandRanges) { + if (leftVars.size() != rightVars.size() || leftVars.size() != bandRanges.size()) + return false; + for (int i = 0; i < bandRanges.size(); i++) { + if (bandRanges.get(i).first != null || bandRanges.get(i).second != null) + return true; + } + return false; + } + + // Currently, we support the int and float/double and will make it general in the future. + public static SortMergeJoinType getSortMergeJoinable(ILogicalOperator op, Collection inLeftAll, + Collection inRightAll, List outLeftFields, + List outRightFields, List> outBandRanges) { + // Three SortMergeJoinable operations: band, theta, metric and skyline. Currently just for band. + ILogicalExpression e = ((AbstractBinaryJoinOperator) op).getCondition().getValue(); + switch (e.getExpressionTag()) { + case FUNCTION_CALL: { + // outBandRanges post process and cut off the band from the Select operator + if (SortMergeJoinType.BAND == getSortMergeJoinable(e, inLeftAll, inRightAll, outLeftFields, + outRightFields, outBandRanges)) + return SortMergeJoinType.BAND; + } + default: + break; + } + return SortMergeJoinType.NESTLOOP; + } + public static boolean isDataSetCall(ILogicalExpression e) { if (((AbstractLogicalExpression) e).getExpressionTag() != LogicalExpressionTag.FUNCTION_CALL) { return false; diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java index 9382b2d06c7..c35d446e1a7 100644 --- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java +++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SetAsterixPhysicalOperatorsRule.java @@ -29,6 +29,7 @@ import org.apache.asterix.metadata.declared.AqlSourceId; import org.apache.asterix.metadata.entities.Dataset; import org.apache.asterix.om.functions.AsterixBuiltinFunctions; +import org.apache.asterix.optimizer.base.AnalysisUtil; import org.apache.asterix.optimizer.rules.am.AccessMethodJobGenParams; import org.apache.asterix.optimizer.rules.am.BTreeJobGenParams; import org.apache.commons.lang3.mutable.Mutable; @@ -44,12 +45,14 @@ import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag; import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable; import org.apache.hyracks.algebricks.core.algebra.base.OperatorAnnotations; +import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag; import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression; import org.apache.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression; import org.apache.hyracks.algebricks.core.algebra.expressions.IMergeAggregationExpressionFactory; import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression; import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier; import org.apache.hyracks.algebricks.core.algebra.metadata.IDataSourceIndex; +import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator; import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator; import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans; import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestMapOperator; @@ -193,6 +196,43 @@ private static void computeDefaultPhysicalOp(AbstractLogicalOperator op, IOptimi } } } + + if (null != op.getPhysicalOperator() + && op.getPhysicalOperator().getOperatorTag() == PhysicalOperatorTag.NESTED_LOOP) { + AbstractBinaryJoinOperator jop = (AbstractBinaryJoinOperator) op; + List sideLeft = new ArrayList(); + List sideRight = new ArrayList(); + List> bandRange = new ArrayList>(); + List varsLeft = op.getInputs().get(0).getValue().getSchema(); + List varsRight = op.getInputs().get(1).getValue().getSchema(); + switch (AnalysisUtil.getSortMergeJoinable(op, varsLeft, varsRight, sideLeft, sideRight, bandRange)) { + case BAND: { + /*op.setPhysicalOperator(new BandSortMergeJoinPOperator(jop.getJoinKind(), + JoinPartitioningType.PAIRWISE, sideLeft, sideRight, bandRange.get(0), + bandRange.get(0).first, context.getPhysicalOptimizationConfig().getMaxFramesHybridHash(), + context.getPhysicalOptimizationConfig().getMaxFramesLeftInputHybridHash(), 40, context + .getPhysicalOptimizationConfig().getMaxRecordsPerFrame(), context + .getPhysicalOptimizationConfig().getFudgeFactor()));*/ + break; + } + case THETA: { + break; + } + case METRIC: { + break; + } + case SKYLINE: { + break; + } + case NESTLOOP: { + break; + } + default: { + break; + } + } + } + if (op.getPhysicalOperator() == null) { switch (op.getOperatorTag()) { case INNERJOIN: { diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java index 82d0b0ef5c9..0adcefddb69 100644 --- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java +++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java @@ -21,6 +21,7 @@ public enum PhysicalOperatorTag { AGGREGATE, ASSIGN, + BAND_SORTMERGE_JOIN, BROADCAST_EXCHANGE, BTREE_SEARCH, BULKLOAD, diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/SortMergeJoinExpressionAnnotation.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/SortMergeJoinExpressionAnnotation.java new file mode 100644 index 00000000000..21991a5b2af --- /dev/null +++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/SortMergeJoinExpressionAnnotation.java @@ -0,0 +1,64 @@ +/* + * 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.hyracks.algebricks.core.algebra.expressions; + +import org.apache.hyracks.algebricks.core.algebra.functions.AlgebricksBuiltinFunctions.ComparisonKind; + +public class SortMergeJoinExpressionAnnotation implements IExpressionAnnotation { + public static final String SORT_MERGE_JOIN_EXPRESSION_ANNOTATION = "smjoin"; + + public static final ComparisonKind[][] thetaMatrix = { { null }, { ComparisonKind.GE, ComparisonKind.GT }, + { ComparisonKind.LE, ComparisonKind.LT }, { ComparisonKind.GE, ComparisonKind.GT }, + { ComparisonKind.LE, ComparisonKind.LT }, { null } }; + + public static final ComparisonKind[] switchMapping = { ComparisonKind.NEQ, ComparisonKind.GT, ComparisonKind.LT, + ComparisonKind.GE, ComparisonKind.LE, ComparisonKind.EQ }; + + public enum SortMergeJoinType { + BAND, + THETA, + METRIC, + SKYLINE, + NESTLOOP + } + + SortMergeJoinType type; + + @Override + public Object getObject() { + // TODO Auto-generated method stub + return type; + } + + @Override + public void setObject(Object object) { + // TODO Auto-generated method stub + this.type = (SortMergeJoinType) object; + } + + @Override + public IExpressionAnnotation copy() { + // TODO Auto-generated method stub + SortMergeJoinExpressionAnnotation smjoin = new SortMergeJoinExpressionAnnotation(); + smjoin.type = type; + return smjoin; + } + +} diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksTaskContext.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksTaskContext.java index 58adccbb39b..14fdfd8b637 100644 --- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksTaskContext.java +++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/context/IHyracksTaskContext.java @@ -22,6 +22,7 @@ import java.util.concurrent.ExecutorService; import org.apache.hyracks.api.dataflow.TaskAttemptId; +import org.apache.hyracks.api.dataflow.state.IStateObject; import org.apache.hyracks.api.dataset.IDatasetPartitionManager; import org.apache.hyracks.api.deployment.DeploymentId; import org.apache.hyracks.api.io.IWorkspaceFileFactory; @@ -40,6 +41,10 @@ public interface IHyracksTaskContext public ExecutorService getExecutorService(); public IDatasetPartitionManager getDatasetPartitionManager(); + + public void setGlobalState(int partition, IStateObject state); + + public IStateObject getGlobalState(int partition); public void sendApplicationMessageToCC(byte[] message, DeploymentId deploymendId) throws Exception; diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputerFactory.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputerFactory.java index cde0057fdc8..91c0a93fb32 100644 --- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputerFactory.java +++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputerFactory.java @@ -20,6 +20,8 @@ import java.io.Serializable; +import org.apache.hyracks.api.context.IHyracksTaskContext; + public interface ITuplePartitionComputerFactory extends Serializable { - public ITuplePartitionComputer createPartitioner(); + public ITuplePartitionComputer createPartitioner(IHyracksTaskContext ctx, int partition); } diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java index 134154c60ef..fa64caaf0c9 100644 --- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java +++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/Task.java @@ -86,6 +86,8 @@ public class Task implements IHyracksTaskContext, ICounterContext, Runnable { private final Set pendingThreads; private IPartitionCollector[] collectors; + + private final static Map stateObj = new HashMap(); private IOperatorNodePushable operator; @@ -402,4 +404,14 @@ public void setSharedObject(Object sharedObject) { public Object getSharedObject() { return sharedObject; } + + @Override + public void setGlobalState(int partition, final IStateObject state) { + Task.stateObj.put(new Integer(partition), state); + } + + @Override + public IStateObject getGlobalState(int partition) { + return Task.stateObj.get(new Integer(partition)); + } } diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java index ee5a041b4fe..0572da95810 100644 --- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java @@ -19,6 +19,7 @@ package org.apache.hyracks.dataflow.common.data.partition; import org.apache.hyracks.api.comm.IFrameTupleAccessor; +import org.apache.hyracks.api.context.IHyracksTaskContext; import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction; import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFactory; import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer; @@ -36,7 +37,7 @@ public FieldHashPartitionComputerFactory(int[] hashFields, IBinaryHashFunctionFa } @Override - public ITuplePartitionComputer createPartitioner() { + public ITuplePartitionComputer createPartitioner(IHyracksTaskContext ctx, int partition) { final IBinaryHashFunction[] hashFunctions = new IBinaryHashFunction[hashFunctionFactories.length]; for (int i = 0; i < hashFunctionFactories.length; ++i) { hashFunctions[i] = hashFunctionFactories[i].createBinaryHashFunction(); diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RandomPartitionComputerFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RandomPartitionComputerFactory.java index e034af07e3b..083c86f4be7 100644 --- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RandomPartitionComputerFactory.java +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RandomPartitionComputerFactory.java @@ -21,6 +21,7 @@ import java.util.Random; import org.apache.hyracks.api.comm.IFrameTupleAccessor; +import org.apache.hyracks.api.context.IHyracksTaskContext; import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer; import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory; import org.apache.hyracks.api.exceptions.HyracksDataException; @@ -29,8 +30,9 @@ public class RandomPartitionComputerFactory implements ITuplePartitionComputerFa private static final long serialVersionUID = 1L; + @Override - public ITuplePartitionComputer createPartitioner() { + public ITuplePartitionComputer createPartitioner(IHyracksTaskContext ctx, int partition) { return new ITuplePartitionComputer() { private final Random random = new Random(); diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFactory.java index 9cb11fab0f9..5465d99ae7d 100644 --- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFactory.java +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFactory.java @@ -19,6 +19,7 @@ package org.apache.hyracks.dataflow.common.data.partition; import org.apache.hyracks.api.comm.IFrameTupleAccessor; +import org.apache.hyracks.api.context.IHyracksTaskContext; import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer; import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory; import org.apache.hyracks.api.exceptions.HyracksDataException; @@ -35,9 +36,9 @@ public RepartitionComputerFactory(int factor, ITuplePartitionComputerFactory del } @Override - public ITuplePartitionComputer createPartitioner() { + public ITuplePartitionComputer createPartitioner(final IHyracksTaskContext ctx, final int partition) { return new ITuplePartitionComputer() { - private ITuplePartitionComputer delegate = delegateFactory.createPartitioner(); + private ITuplePartitionComputer delegate = delegateFactory.createPartitioner(ctx, partition); @Override public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) throws HyracksDataException { diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java index f4da9bfdb20..2d4c0e98ee8 100644 --- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java @@ -19,6 +19,7 @@ package org.apache.hyracks.dataflow.common.data.partition.range; import org.apache.hyracks.api.comm.IFrameTupleAccessor; +import org.apache.hyracks.api.context.IHyracksTaskContext; import org.apache.hyracks.api.dataflow.value.IBinaryComparator; import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory; import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer; @@ -39,7 +40,7 @@ public FieldRangePartitionComputerFactory(int[] rangeFields, IBinaryComparatorFa } @Override - public ITuplePartitionComputer createPartitioner() { + public ITuplePartitionComputer createPartitioner(IHyracksTaskContext ctx, int partition) { final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length]; for (int i = 0; i < comparatorFactories.length; ++i) { comparators[i] = comparatorFactories[i].createBinaryComparator(); diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopHashTuplePartitionComputerFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopHashTuplePartitionComputerFactory.java index fe8a612e38b..fc9dabbbfd2 100644 --- a/hyracks-fullstack/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopHashTuplePartitionComputerFactory.java +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopHashTuplePartitionComputerFactory.java @@ -21,8 +21,8 @@ import java.io.DataInputStream; import org.apache.hadoop.io.Writable; - import org.apache.hyracks.api.comm.IFrameTupleAccessor; +import org.apache.hyracks.api.context.IHyracksTaskContext; import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer; import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer; import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory; @@ -38,7 +38,7 @@ public HadoopHashTuplePartitionComputerFactory(ISerializerDeserializer keyIO) } @Override - public ITuplePartitionComputer createPartitioner() { + public ITuplePartitionComputer createPartitioner(IHyracksTaskContext ctx, int partition) { return new ITuplePartitionComputer() { private final ByteBufferInputStream bbis = new ByteBufferInputStream(); private final DataInputStream dis = new DataInputStream(bbis); diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopNewPartitionerTuplePartitionComputerFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopNewPartitionerTuplePartitionComputerFactory.java index b20c6e0a8ea..b6dc967682c 100644 --- a/hyracks-fullstack/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopNewPartitionerTuplePartitionComputerFactory.java +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-hadoop/src/main/java/org/apache/hyracks/dataflow/hadoop/data/HadoopNewPartitionerTuplePartitionComputerFactory.java @@ -22,8 +22,8 @@ import org.apache.hadoop.io.Writable; import org.apache.hadoop.mapreduce.Partitioner; - import org.apache.hyracks.api.comm.IFrameTupleAccessor; +import org.apache.hyracks.api.context.IHyracksTaskContext; import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer; import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer; import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory; @@ -44,7 +44,7 @@ public HadoopNewPartitionerTuplePartitionComputerFactory(Class mConfig; - public HashPartitioningShuffleConnectorDescriptor(IConnectorDescriptorRegistry spec, MarshalledWritable mConfig) { + public HashPartitioningShuffleConnectorDescriptor(IConnectorDescriptorRegistry spec, + MarshalledWritable mConfig) { super(spec); this.mConfig = mConfig; } @@ -52,7 +53,8 @@ public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor throws HyracksDataException { HadoopHelper helper = new HadoopHelper(mConfig); ITuplePartitionComputerFactory tpcf = helper.getTuplePartitionComputer(); - return new PartitionDataWriter(ctx, nConsumerPartitions, edwFactory, recordDesc, tpcf.createPartitioner()); + return new PartitionDataWriter(ctx, nConsumerPartitions, edwFactory, recordDesc, tpcf.createPartitioner(ctx, + index)); } @Override diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/LocalityAwareMToNPartitioningConnectorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/LocalityAwareMToNPartitioningConnectorDescriptor.java index 44d77ac99e5..a398ab92af2 100644 --- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/LocalityAwareMToNPartitioningConnectorDescriptor.java +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/LocalityAwareMToNPartitioningConnectorDescriptor.java @@ -60,8 +60,8 @@ public LocalityAwareMToNPartitioningConnectorDescriptor(IConnectorDescriptorRegi @Override public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc, IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions) - throws HyracksDataException { - return new LocalityAwarePartitionDataWriter(ctx, edwFactory, recordDesc, tpcf.createPartitioner(), + throws HyracksDataException { + return new LocalityAwarePartitionDataWriter(ctx, edwFactory, recordDesc, tpcf.createPartitioner(ctx, index), nConsumerPartitions, localityMap, index); } diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningConnectorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningConnectorDescriptor.java index 4872b95eac4..5674a9ee574 100644 --- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningConnectorDescriptor.java +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningConnectorDescriptor.java @@ -45,13 +45,15 @@ public MToNPartitioningConnectorDescriptor(IConnectorDescriptorRegistry spec, IT @Override public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc, IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions) - throws HyracksDataException { - return new PartitionDataWriter(ctx, nConsumerPartitions, edwFactory, recordDesc, tpcf.createPartitioner()); + throws HyracksDataException { + final PartitionDataWriter hashWriter = new PartitionDataWriter(ctx, nConsumerPartitions, edwFactory, + recordDesc, tpcf.createPartitioner(ctx, index)); + return hashWriter; } @Override - public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc, int index, - int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException { + public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc, + int index, int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException { BitSet expectedPartitions = new BitSet(nProducerPartitions); expectedPartitions.set(0, nProducerPartitions); NonDeterministicChannelReader channelReader = new NonDeterministicChannelReader(nProducerPartitions, diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java index 04de894399f..dfca19234bd 100644 --- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java @@ -70,7 +70,7 @@ public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException { final PartitionDataWriter hashWriter = new PartitionDataWriter(ctx, nConsumerPartitions, edwFactory, - recordDesc, tpcf.createPartitioner()); + recordDesc, tpcf.createPartitioner(ctx, index)); return hashWriter; } diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningWithMessageConnectorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningWithMessageConnectorDescriptor.java index e90d8b0ad10..76fbe3e2eee 100644 --- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningWithMessageConnectorDescriptor.java +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningWithMessageConnectorDescriptor.java @@ -40,6 +40,6 @@ public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException { return new PartitionWithMessageDataWriter(ctx, nConsumerPartitions, edwFactory, recordDesc, - tpcf.createPartitioner()); + tpcf.createPartitioner(ctx, index)); } } diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/GraceHashJoinOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/GraceHashJoinOperatorNodePushable.java index 69e9e6a1451..8899a655745 100644 --- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/GraceHashJoinOperatorNodePushable.java +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/GraceHashJoinOperatorNodePushable.java @@ -94,9 +94,9 @@ public void initialize() throws HyracksDataException { comparators[i] = comparatorFactories[i].createBinaryComparator(); } ITuplePartitionComputer hpcRep0 = new RepartitionComputerFactory(numPartitions, - new FieldHashPartitionComputerFactory(keys0, hashFunctionFactories)).createPartitioner(); + new FieldHashPartitionComputerFactory(keys0, hashFunctionFactories)).createPartitioner(ctx, -1); ITuplePartitionComputer hpcRep1 = new RepartitionComputerFactory(numPartitions, - new FieldHashPartitionComputerFactory(keys1, hashFunctionFactories)).createPartitioner(); + new FieldHashPartitionComputerFactory(keys1, hashFunctionFactories)).createPartitioner(ctx, -1); final INullWriter[] nullWriters1 = isLeftOuter ? new INullWriter[nullWriterFactories.length] : null; if (isLeftOuter) { diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/GraceHashJoinPartitionBuildOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/GraceHashJoinPartitionBuildOperatorNodePushable.java index 5a5543ba759..4c49c45cbbc 100644 --- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/GraceHashJoinPartitionBuildOperatorNodePushable.java +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/GraceHashJoinPartitionBuildOperatorNodePushable.java @@ -36,8 +36,7 @@ import org.apache.hyracks.dataflow.common.io.RunFileWriter; import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable; -class GraceHashJoinPartitionBuildOperatorNodePushable extends - AbstractUnaryInputSinkOperatorNodePushable { +class GraceHashJoinPartitionBuildOperatorNodePushable extends AbstractUnaryInputSinkOperatorNodePushable { private final IHyracksTaskContext ctx; private final Object stateId; private final int numPartitions; @@ -56,7 +55,7 @@ class GraceHashJoinPartitionBuildOperatorNodePushable extends this.numPartitions = numPartitions; accessor0 = new FrameTupleAccessor(inRecordDescriptor); appender = new FrameTupleAppender(); - hpc = new FieldHashPartitionComputerFactory(keys, hashFunctionFactories).createPartitioner(); + hpc = new FieldHashPartitionComputerFactory(keys, hashFunctionFactories).createPartitioner(ctx, -1); comparators = new IBinaryComparator[comparatorFactories.length]; for (int i = 0; i < comparatorFactories.length; ++i) { comparators[i] = comparatorFactories[i].createBinaryComparator(); diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java index d0a81ee5a33..faa8ef653c8 100644 --- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java @@ -190,7 +190,7 @@ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, ctx.getJobletContext().getJobId(), new TaskId(getActivityId(), partition)); private final FrameTupleAccessor accessorBuild = new FrameTupleAccessor(rd1); private final ITuplePartitionComputer hpcBuild = new FieldHashPartitionComputerFactory(keys1, - hashFunctionFactories).createPartitioner(); + hashFunctionFactories).createPartitioner(ctx, partition); private final FrameTupleAppender appender = new FrameTupleAppender(); private final FrameTupleAppender ftappender = new FrameTupleAppender(); private IFrame[] bufferForPartitions; @@ -302,9 +302,9 @@ public void open() throws HyracksDataException { } ITuplePartitionComputer hpc0 = new FieldHashPartitionComputerFactory(keys0, hashFunctionFactories) - .createPartitioner(); + .createPartitioner(ctx, -1); ITuplePartitionComputer hpc1 = new FieldHashPartitionComputerFactory(keys1, hashFunctionFactories) - .createPartitioner(); + .createPartitioner(ctx, -1); int tableSize = (int) (state.memoryForHashtable * recordsPerFrame * factor); ISerializableTable table = new SerializableHashTable(tableSize, ctx); state.joiner = new InMemoryHashJoin(ctx, tableSize, new FrameTupleAccessor(rd0), hpc0, @@ -382,7 +382,7 @@ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, hashFunctionFactories); private final ITuplePartitionComputerFactory hpcf1 = new FieldHashPartitionComputerFactory(keys1, hashFunctionFactories); - private final ITuplePartitionComputer hpcProbe = hpcf0.createPartitioner(); + private final ITuplePartitionComputer hpcProbe = hpcf0.createPartitioner(ctx, partition); private final FrameTupleAppender appender = new FrameTupleAppender(); private final FrameTupleAppender ftap = new FrameTupleAppender(); @@ -468,9 +468,9 @@ public void close() throws HyracksDataException { state.joiner.join(inBuffer.getBuffer(), writer); state.joiner.closeJoin(writer); ITuplePartitionComputer hpcRep0 = new RepartitionComputerFactory(state.nPartitions, hpcf0) - .createPartitioner(); + .createPartitioner(ctx, -1); ITuplePartitionComputer hpcRep1 = new RepartitionComputerFactory(state.nPartitions, hpcf1) - .createPartitioner(); + .createPartitioner(ctx, -1); if (state.memoryForHashtable != memsize - 2) { for (int i = 0; i < state.nPartitions; i++) { ByteBuffer buf = bufferForPartitions[i].getBuffer(); @@ -488,6 +488,7 @@ public void close() throws HyracksDataException { } else { tableSize = (int) (memsize * recordsPerFrame * factor); } + ISerializableTable table = new SerializableHashTable(tableSize, ctx); for (int partitionid = 0; partitionid < state.nPartitions; partitionid++) { RunFileWriter buildWriter = buildWriters[partitionid]; diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java index be8d319f2b5..4ee5965d2da 100644 --- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java @@ -182,9 +182,9 @@ public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, @Override public void open() throws HyracksDataException { ITuplePartitionComputer hpc0 = new FieldHashPartitionComputerFactory(keys0, hashFunctionFactories) - .createPartitioner(); + .createPartitioner(ctx, -1); ITuplePartitionComputer hpc1 = new FieldHashPartitionComputerFactory(keys1, hashFunctionFactories) - .createPartitioner(); + .createPartitioner(ctx, -1); state = new HashBuildTaskState(ctx.getJobletContext().getJobId(), new TaskId(getActivityId(), partition)); ISerializableTable table = new SerializableHashTable(tableSize, ctx); diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/HistogramAlgorithm.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/HistogramAlgorithm.java new file mode 100644 index 00000000000..a0134f3e380 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/HistogramAlgorithm.java @@ -0,0 +1,30 @@ +/* + * 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.hyracks.dataflow.std.parallel; + +/** + * @author michael + */ +public enum HistogramAlgorithm { + ORDERED_HISTOGRAM, // For the local-sorted fields. + UNIFORM_HISTOGRAM, // For the equal-height histogram. + RANDOM_HISTOGRAM, // For the sample based rangeMap. + WAVELET_HISTOGRAM // For the wavelet histogram. +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/IDTHistogram.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/IDTHistogram.java new file mode 100644 index 00000000000..f1b29fe832a --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/IDTHistogram.java @@ -0,0 +1,30 @@ +/* + * 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.hyracks.dataflow.std.parallel; + +import org.apache.hyracks.api.exceptions.HyracksDataException; + +/** + * @author michael + */ +public interface IDTHistogram extends IHistogram { + + public E mediate(E left, E right) throws HyracksDataException; +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/IHistogram.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/IHistogram.java new file mode 100644 index 00000000000..3f5134ff92d --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/IHistogram.java @@ -0,0 +1,56 @@ +/* + * 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.hyracks.dataflow.std.parallel; + +import java.util.List; +import java.util.Map.Entry; + +import org.apache.hyracks.api.exceptions.HyracksDataException; + +/** + * @author michael + */ +public interface IHistogram { + + public enum FieldType { + SHORT, + INT, + LONG, + FLOAT, + DOUBLE, + UTF8 + } + + public void initialize(); + + public FieldType getType(); + + public void merge(IHistogram ba) throws HyracksDataException; + + public void addItem(E item) throws HyracksDataException; + + public void countItem(E item) throws HyracksDataException; + + public void countReset() throws HyracksDataException; + + public int getCurrent() throws HyracksDataException; + + public List> generate(boolean isGlobal) throws HyracksDataException; +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/IIterativeHistogram.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/IIterativeHistogram.java new file mode 100644 index 00000000000..accc32402c0 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/IIterativeHistogram.java @@ -0,0 +1,39 @@ +/* + * 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.hyracks.dataflow.std.parallel; + +import java.util.List; +import java.util.Map.Entry; + +import org.apache.hyracks.api.exceptions.HyracksDataException; + +/** + * @author michael + */ +public interface IIterativeHistogram extends IHistogram { + + public void disperse(/*E path, int incLevel*/) throws HyracksDataException; + + public boolean needIteration() throws HyracksDataException; + + public boolean isFixPointable() throws HyracksDataException; + + public List> getFixPointable() throws HyracksDataException; +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/ISequentialAccessor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/ISequentialAccessor.java new file mode 100644 index 00000000000..f04b3017796 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/ISequentialAccessor.java @@ -0,0 +1,42 @@ +/* + * 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.hyracks.dataflow.std.parallel; + +/** + * @author michael + */ +public interface ISequentialAccessor { + + public int cursor(); + + public char first(); + + public char current(); + + public char next(); + + public void reset(); + + public char at(int i); + + public int length(); + + public String toString(); +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/ISequentialTrie.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/ISequentialTrie.java new file mode 100644 index 00000000000..daa1df1cdfe --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/ISequentialTrie.java @@ -0,0 +1,33 @@ +/* + * 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.hyracks.dataflow.std.parallel; + +public interface ISequentialTrie { + public boolean insert(E p, int id); + + public int search(E p); + + public boolean delete(E p); + + // Should be deprecated in the near future. + public void serialize(short limit); + + public E next(); +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/IStatisticEntity.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/IStatisticEntity.java new file mode 100644 index 00000000000..faba83842f2 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/IStatisticEntity.java @@ -0,0 +1,33 @@ +/* + * 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.hyracks.dataflow.std.parallel; + +/** + * @author michael + */ +public interface IStatisticEntity { + public int getCount(E e); + + public void clearCount(E e); + + public void updateBy(E e, int ub); + + public void grow(E e, boolean deeper, short limit); +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/IStatisticNode.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/IStatisticNode.java new file mode 100644 index 00000000000..3d1e7a340d6 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/IStatisticNode.java @@ -0,0 +1,32 @@ +/* + * 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.hyracks.dataflow.std.parallel; + +/** + * @author michael + */ +public interface IStatisticNode { + + public int getCount(); + + public void clearCount(); + + public void updateBy(int ub); +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/IStatisticTernaryNode.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/IStatisticTernaryNode.java new file mode 100644 index 00000000000..72bcda08c04 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/IStatisticTernaryNode.java @@ -0,0 +1,63 @@ +/* + * 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.hyracks.dataflow.std.parallel; + +/** + * @author michael + */ +public interface IStatisticTernaryNode extends IStatisticNode { + public int getId(); + + public void setId(int id); + + public short getLimit(); + + public void setLimit(int limit); + + public short getLevel(); + + public void setLevel(int limit); + + public char getKey(); + + public void setKey(char key); + + public boolean isActive(); + + public IStatisticTernaryNode getLeft(); + + public void setLeft(IStatisticTernaryNode left); + + public IStatisticTernaryNode getRight(); + + public void setRight(IStatisticTernaryNode right); + + public IStatisticTernaryNode getMiddle(); + + public void setMiddle(IStatisticTernaryNode middle); + + public void setPayload(T payload); + + public T getPayload(); + + public void setGrown(); + + public boolean isGrown(); +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/AbstractSamplingWriter.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/AbstractSamplingWriter.java new file mode 100644 index 00000000000..009bb0948f5 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/AbstractSamplingWriter.java @@ -0,0 +1,228 @@ +/* + * 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.hyracks.dataflow.std.parallel.base; + +import java.nio.ByteBuffer; +import java.util.logging.Logger; + +import org.apache.hyracks.api.comm.IFrame; +import org.apache.hyracks.api.comm.IFrameTupleAccessor; +import org.apache.hyracks.api.comm.IFrameWriter; +import org.apache.hyracks.api.comm.VSizeFrame; +import org.apache.hyracks.api.context.IHyracksTaskContext; +import org.apache.hyracks.api.dataflow.value.IBinaryComparator; +import org.apache.hyracks.api.dataflow.value.RecordDescriptor; +import org.apache.hyracks.api.exceptions.HyracksDataException; +import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder; +import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor; +import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppender; +import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAppenderWrapper; +import org.apache.hyracks.dataflow.common.comm.util.FrameUtils; +import org.apache.hyracks.dataflow.std.group.AggregateState; +import org.apache.hyracks.dataflow.std.group.IFieldAggregateDescriptor; +import org.apache.hyracks.dataflow.std.group.aggregators.CountFieldAggregatorFactory; + +/** + * @author michael + */ +public abstract class AbstractSamplingWriter implements IFrameWriter { + private static final Logger LOGGER = Logger.getLogger(AbstractSamplingWriter.class.getName()); + // private final IHyracksTaskContext ctx; + // private final SampleAlgorithm alg = SampleAlgorithm.ORDERED_SAMPLE; + protected final FrameTupleAppenderWrapper appenderWrapper; + protected boolean outputPartial = false; + + protected boolean isFailed = false; + protected final int[] sampleFields; + protected int sampleBasis; + protected final IBinaryComparator[] comparators; + protected final IFrame copyFrame; + protected final FrameTupleAccessor inFrameAccessor; + protected final FrameTupleAccessor copyFrameAccessor; + protected IFieldAggregateDescriptor aggregator; + protected AggregateState aggregateState; + protected final ArrayTupleBuilder tupleBuilder; + protected boolean isFirst; + + public AbstractSamplingWriter(IHyracksTaskContext ctx, int[] sampleFields, int sampleBasis, + IBinaryComparator[] comparators, RecordDescriptor inRecordDesc, RecordDescriptor outRecordDesc, + IFrameWriter writer, boolean outputPartial) throws HyracksDataException { + this(ctx, sampleFields, sampleBasis, comparators, inRecordDesc, outRecordDesc, writer); + this.outputPartial = outputPartial; + } + + public AbstractSamplingWriter(IHyracksTaskContext ctx, int[] sampleFields, int sampleBasis, + IBinaryComparator[] comparators, RecordDescriptor inRecordDesc, RecordDescriptor outRecordDesc, + IFrameWriter writer) throws HyracksDataException { + // this.ctx = ctx; + this.sampleFields = sampleFields; + this.sampleBasis = sampleBasis; + this.comparators = comparators; + copyFrame = new VSizeFrame(ctx); + inFrameAccessor = new FrameTupleAccessor(inRecordDesc); + copyFrameAccessor = new FrameTupleAccessor(inRecordDesc); + copyFrameAccessor.reset(copyFrame.getBuffer()); + aggregator = new CountFieldAggregatorFactory(true).createAggregator(ctx, inRecordDesc, outRecordDesc); + aggregateState = aggregator.createState(); + + VSizeFrame outFrame = new VSizeFrame(ctx); + FrameTupleAppender appender = new FrameTupleAppender(); + appender.reset(outFrame, true); + appenderWrapper = new FrameTupleAppenderWrapper(appender, writer); + + tupleBuilder = new ArrayTupleBuilder(outRecordDesc.getFields().length); + } + + @Override + public void open() throws HyracksDataException { + appenderWrapper.open(); + isFirst = true; + } + + @Override + public void nextFrame(ByteBuffer buffer) throws HyracksDataException { + inFrameAccessor.reset(buffer); + int nTuples = inFrameAccessor.getTupleCount(); + /*switch (alg) { + case ORDERED_SAMPLE: {*/ + for (int i = 0; i < nTuples; i++) { + if (isFirst) { + tupleBuilder.reset(); + for (int j = 0; j < sampleFields.length; j++) { + tupleBuilder.addField(inFrameAccessor, i, sampleFields[j]); + } + aggregator.init(inFrameAccessor, i, tupleBuilder.getDataOutput(), aggregateState); + isFirst = false; + } else { + // each frame need to be at least sampled once. + if (i == 0) { + switchBinsIfRequired(copyFrameAccessor, copyFrameAccessor.getTupleCount() - 1, inFrameAccessor, i); + } else { + try { + switchBinsIfRequired(inFrameAccessor, i - 1, inFrameAccessor, i); + } catch (Exception e) { + LOGGER.info("Sampling error: " + tupleBuilder.getDataOutput().getClass().getName()); + throw new HyracksDataException("Failed to get the proper sampling bins"); + } + } + } + } + copyFrame.ensureFrameSize(buffer.capacity()); + FrameUtils.copyAndFlip(buffer, copyFrame.getBuffer()); + copyFrameAccessor.reset(copyFrame.getBuffer()); + /*break; + } + case RANDOM_SAMPLE: + case UNIFORM_SAMPLE: + case WAVELET_SAMPLE: + default: + break; + }*/ + } + + protected boolean writeFieldsOutput(final IFrameTupleAccessor lastTupleAccessor, int lastTupleIndex) + throws HyracksDataException { + int tupleOffset = lastTupleAccessor.getTupleStartOffset(lastTupleIndex); + + if (outputPartial) { + int fieldOffset = lastTupleAccessor.getFieldStartOffset(lastTupleIndex, sampleFields.length); + aggregator.outputPartialResult(tupleBuilder.getDataOutput(), lastTupleAccessor.getBuffer().array(), + tupleOffset + fieldOffset + lastTupleAccessor.getFieldSlotsLength(), aggregateState); + tupleBuilder.addFieldEndOffset(); + } else { + if (aggregator.needsBinaryState()) { + int fieldOffset = lastTupleAccessor.getFieldStartOffset(lastTupleIndex, sampleFields.length); + aggregator.outputFinalResult(tupleBuilder.getDataOutput(), lastTupleAccessor.getBuffer().array(), + tupleOffset + fieldOffset + lastTupleAccessor.getFieldSlotsLength(), aggregateState); + } else { + /*int fieldOffset = lastTupleAccessor.getFieldStartOffset(lastTupleIndex, sampleFields.length); + aggregator.outputFinalResult(tupleBuilder.getDataOutput(), lastTupleAccessor.getBuffer().array(), + tupleOffset + fieldOffset + lastTupleAccessor.getFieldSlotsLength(), aggregateState);*/ + aggregator.outputFinalResult(tupleBuilder.getDataOutput(), null, 0, aggregateState); + } + tupleBuilder.addFieldEndOffset(); + } + return true; + } + + protected void writeOutput(final IFrameTupleAccessor lastTupleAccessor, int lastTupleIndex) + throws HyracksDataException { + tupleBuilder.reset(); + for (int j = 0; j < sampleFields.length; j++) { + tupleBuilder.addField(lastTupleAccessor, lastTupleIndex, sampleFields[j]); + } + boolean hasOutput = writeFieldsOutput(lastTupleAccessor, lastTupleIndex); + if (hasOutput) { + appenderWrapper.appendSkipEmptyField(tupleBuilder.getFieldEndOffsets(), tupleBuilder.getByteArray(), 0, + tupleBuilder.getSize()); + } + } + + protected boolean aggregatingWithBalanceGuaranteed(IFrameTupleAccessor prevTupleAccessor, int prevTupleIndex, + IFrameTupleAccessor currTupleAccessor, int currTupleIndex) throws HyracksDataException { + for (int i = 0; i < comparators.length; i++) { + int fIdx = sampleFields[i]; + int s1 = prevTupleAccessor.getAbsoluteFieldStartOffset(prevTupleIndex, fIdx); + int l1 = prevTupleAccessor.getFieldLength(prevTupleIndex, fIdx); + int s2 = currTupleAccessor.getAbsoluteFieldStartOffset(currTupleIndex, fIdx); + int l2 = currTupleAccessor.getFieldLength(currTupleIndex, fIdx); + if (0 != comparators[i].compare(prevTupleAccessor.getBuffer().array(), s1, l1, currTupleAccessor + .getBuffer().array(), s2, l2)) { + return false; + } + } + return true; + } + + protected void switchBinsIfRequired(IFrameTupleAccessor prevTupleAccessor, int prevTupleIndex, + IFrameTupleAccessor currTupleAccessor, int currTupleIndex) throws HyracksDataException { + if (!aggregatingWithBalanceGuaranteed(prevTupleAccessor, prevTupleIndex, currTupleAccessor, currTupleIndex)) { + writeOutput(prevTupleAccessor, prevTupleIndex); + tupleBuilder.reset(); + for (int i = 0; i < sampleFields.length; i++) { + tupleBuilder.addField(currTupleAccessor, currTupleIndex, sampleFields[i]); + } + aggregator.init(currTupleAccessor, currTupleIndex, tupleBuilder.getDataOutput(), aggregateState); + } else { + try { + aggregator.aggregate(currTupleAccessor, currTupleIndex, null, 0, aggregateState); + } catch (Exception e) { + LOGGER.info("Sampling error: " + tupleBuilder.getDataOutput().getClass().getName()); + throw new HyracksDataException("Failed to get the proper sampling bins"); + } + } + } + + @Override + public void fail() throws HyracksDataException { + isFailed = true; + appenderWrapper.fail(); + } + + @Override + public void close() throws HyracksDataException { + if (!isFailed && !isFirst) { + appenderWrapper.flush(); + } + aggregator.close(); + aggregateState.close(); + appenderWrapper.close(); + } +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/DefaultSamplingWriter.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/DefaultSamplingWriter.java new file mode 100644 index 00000000000..86df2d7b346 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/DefaultSamplingWriter.java @@ -0,0 +1,76 @@ +/* + * 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.hyracks.dataflow.std.parallel.base; + +import org.apache.hyracks.api.comm.IFrameWriter; +import org.apache.hyracks.api.context.IHyracksTaskContext; +import org.apache.hyracks.api.dataflow.value.IBinaryComparator; +import org.apache.hyracks.api.dataflow.value.RecordDescriptor; +import org.apache.hyracks.api.exceptions.HyracksDataException; + +/** + * @author michael + */ +public class DefaultSamplingWriter extends AbstractSamplingWriter { + + /** + * @param ctx + * @param sampleFields + * @param sampleBasis + * @param comparators + * @param inRecordDesc + * @param outRecordDesc + * @param writer + * @param outputPartial + * @throws HyracksDataException + */ + public DefaultSamplingWriter(IHyracksTaskContext ctx, int[] sampleFields, int sampleBasis, + IBinaryComparator[] comparators, RecordDescriptor inRecordDesc, RecordDescriptor outRecordDesc, + IFrameWriter writer, boolean outputPartial) throws HyracksDataException { + super(ctx, sampleFields, sampleBasis, comparators, inRecordDesc, outRecordDesc, writer, outputPartial); + // TODO Auto-generated constructor stub + } + + /** + * @param ctx + * @param sampleFields + * @param sampleBasis + * @param comparators + * @param inRecordDesc + * @param outRecordDesc + * @param writer + * @throws HyracksDataException + */ + public DefaultSamplingWriter(IHyracksTaskContext ctx, int[] sampleFields, int sampleBasis, + IBinaryComparator[] comparators, RecordDescriptor inRecordDesc, RecordDescriptor outRecordDesc, + IFrameWriter writer) throws HyracksDataException { + super(ctx, sampleFields, sampleBasis, comparators, inRecordDesc, outRecordDesc, writer); + // TODO Auto-generated constructor stub + } + + @Override + public void close() throws HyracksDataException { + if (!isFailed && !isFirst) { + assert (copyFrameAccessor.getTupleCount() > 0); + writeOutput(copyFrameAccessor, copyFrameAccessor.getTupleCount() - 1); + } + super.close(); + } +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/FieldRangePartitionDelayComputerFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/FieldRangePartitionDelayComputerFactory.java new file mode 100644 index 00000000000..912857fad5c --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/FieldRangePartitionDelayComputerFactory.java @@ -0,0 +1,117 @@ +/* + * 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.hyracks.dataflow.std.parallel.base; + +import org.apache.hyracks.api.comm.IFrameTupleAccessor; +import org.apache.hyracks.api.context.IHyracksTaskContext; +import org.apache.hyracks.api.dataflow.state.IStateObject; +import org.apache.hyracks.api.dataflow.value.IBinaryComparator; +import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory; +import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer; +import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory; +import org.apache.hyracks.api.exceptions.HyracksDataException; +import org.apache.hyracks.dataflow.common.data.partition.range.IRangeMap; + +/** + * @author michael + */ +public class FieldRangePartitionDelayComputerFactory implements ITuplePartitionComputerFactory { + + private static final long serialVersionUID = 1L; + private final int[] rangeFields; + private IRangeMap rangeMap; + private IBinaryComparatorFactory[] comparatorFactories; + + public FieldRangePartitionDelayComputerFactory(int[] rangeFields, IBinaryComparatorFactory[] comparatorFactories) { + this.rangeFields = rangeFields; + this.comparatorFactories = comparatorFactories; + } + + @Override + public ITuplePartitionComputer createPartitioner(IHyracksTaskContext ctx, int index) { + + try { + IStateObject rangeState = ctx.getGlobalState(index); + rangeMap = ((ParallelRangeMapTaskState) rangeState).getRangeMap(); + } catch (HyracksDataException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + + final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length]; + for (int i = 0; i < comparatorFactories.length; ++i) { + comparators[i] = comparatorFactories[i].createBinaryComparator(); + } + return new ITuplePartitionComputer() { + @Override + /** + * Determine the range partition. + */ + public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) throws HyracksDataException { + if (nParts == 1) { + return 0; + } + int slotIndex = getRangePartition(accessor, tIndex); + // Map range partition to node partitions. + double rangesPerPart = 1; + if (rangeMap.getSplitCount() + 1 > nParts) { + rangesPerPart = ((double) rangeMap.getSplitCount() + 1) / nParts; + } + return (int) Math.floor(slotIndex / rangesPerPart); + } + + /* + * Determine the range partition. + */ + public int getRangePartition(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException { + int slotIndex = 0; + for (int i = 0; i < rangeMap.getSplitCount(); ++i) { + int c = compareSlotAndFields(accessor, tIndex, i); + if (c < 0) { + return slotIndex; + } + slotIndex++; + } + return slotIndex; + } + + public int compareSlotAndFields(IFrameTupleAccessor accessor, int tIndex, int fieldIndex) + throws HyracksDataException { + int c = 0; + int startOffset = accessor.getTupleStartOffset(tIndex); + int slotLength = accessor.getFieldSlotsLength(); + for (int f = 0; f < comparators.length; ++f) { + int fIdx = rangeFields[f]; + int fStart = accessor.getFieldStartOffset(tIndex, fIdx); + int fEnd = accessor.getFieldEndOffset(tIndex, fIdx); + c = comparators[f].compare(accessor.getBuffer().array(), startOffset + slotLength + fStart, fEnd + - fStart, rangeMap.getByteArray(fieldIndex, f), rangeMap.getStartOffset(fieldIndex, f), + rangeMap.getLength(fieldIndex, f)); + if (c != 0) { + return c; + } + } + return c; + } + + }; + } + +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/HistogramConnectorPolicyAssignmentPolicy.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/HistogramConnectorPolicyAssignmentPolicy.java new file mode 100644 index 00000000000..bc4d8cf98fe --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/HistogramConnectorPolicyAssignmentPolicy.java @@ -0,0 +1,43 @@ +/* + * 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.hyracks.dataflow.std.parallel.base; + +import org.apache.hyracks.api.dataflow.IConnectorDescriptor; +import org.apache.hyracks.api.dataflow.connectors.IConnectorPolicy; +import org.apache.hyracks.api.dataflow.connectors.IConnectorPolicyAssignmentPolicy; +import org.apache.hyracks.api.dataflow.connectors.PipeliningConnectorPolicy; +import org.apache.hyracks.api.dataflow.connectors.SendSideMaterializedPipeliningConnectorPolicy; +import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor; + +public class HistogramConnectorPolicyAssignmentPolicy implements IConnectorPolicyAssignmentPolicy { + private static final long serialVersionUID = 1L; + private IConnectorPolicy senderSideMaterializePolicy = new SendSideMaterializedPipeliningConnectorPolicy(); + private IConnectorPolicy pipeliningPolicy = new PipeliningConnectorPolicy(); + + @Override + public IConnectorPolicy getConnectorPolicyAssignment(IConnectorDescriptor c, int nProducers, int nConsumers, + int[] fanouts) { + if (c instanceof MToNPartitioningMergingConnectorDescriptor) { + return senderSideMaterializePolicy; + } else { + return pipeliningPolicy; + } + } +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/MaterializingSampleTaskState.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/MaterializingSampleTaskState.java new file mode 100644 index 00000000000..823a009376b --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/MaterializingSampleTaskState.java @@ -0,0 +1,90 @@ +/* + * 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.hyracks.dataflow.std.parallel.base; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.nio.ByteBuffer; + +import org.apache.hyracks.api.comm.IFrame; +import org.apache.hyracks.api.comm.IFrameWriter; +import org.apache.hyracks.api.context.IHyracksTaskContext; +import org.apache.hyracks.api.exceptions.HyracksDataException; +import org.apache.hyracks.api.io.FileReference; +import org.apache.hyracks.api.job.JobId; +import org.apache.hyracks.dataflow.common.io.RunFileReader; +import org.apache.hyracks.dataflow.common.io.RunFileWriter; +import org.apache.hyracks.dataflow.std.base.AbstractStateObject; + +public class MaterializingSampleTaskState extends AbstractStateObject { + private RunFileWriter out; + + public MaterializingSampleTaskState(JobId jobId, Object taskId) { + super(jobId, taskId); + } + + @Override + public void toBytes(DataOutput out) throws IOException { + // TODO Auto-generated method stub + + } + + @Override + public void fromBytes(DataInput in) throws IOException { + // TODO Auto-generated method stub + } + + public void open(IHyracksTaskContext context) throws HyracksDataException { + FileReference file = context.getJobletContext().createManagedWorkspaceFile( + MaterializingSampleTaskState.class.getSimpleName()); + out = new RunFileWriter(file, context.getIOManager()); + out.open(); + } + + public void close() throws HyracksDataException { + out.close(); + } + + public void appendFrame(ByteBuffer buf) throws HyracksDataException { + out.nextFrame(buf); + } + + public void writeOut(IFrameWriter writer, IFrame frame) throws HyracksDataException { + RunFileReader in = out.createReader(); + writer.open(); + try { + in.open(); + while (in.nextFrame(frame)) { + writer.nextFrame(frame.getBuffer()); + } + in.close(); + } catch (Exception e) { + writer.fail(); + throw new HyracksDataException(e); + } finally { + writer.close(); + } + } + + public void deleteFile() { + out.getFileReference().delete(); + } +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/MergeOrderedSampleWriter.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/MergeOrderedSampleWriter.java new file mode 100644 index 00000000000..18db4e7a98c --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/MergeOrderedSampleWriter.java @@ -0,0 +1,177 @@ +/* + * 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.hyracks.dataflow.std.parallel.base; + +import java.nio.ByteBuffer; + +import org.apache.hyracks.api.comm.IFrame; +import org.apache.hyracks.api.comm.IFrameTupleAccessor; +import org.apache.hyracks.api.comm.IFrameWriter; +import org.apache.hyracks.api.comm.VSizeFrame; +import org.apache.hyracks.api.context.IHyracksTaskContext; +import org.apache.hyracks.api.dataflow.value.IBinaryComparator; +import org.apache.hyracks.api.dataflow.value.RecordDescriptor; +import org.apache.hyracks.api.exceptions.HyracksDataException; +import org.apache.hyracks.data.std.primitive.IntegerPointable; +import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor; +import org.apache.hyracks.dataflow.common.comm.util.FrameUtils; +import org.apache.hyracks.dataflow.std.group.aggregators.IntSumFieldAggregatorFactory; + +/** + * @author michael + */ +public class MergeOrderedSampleWriter extends AbstractSamplingWriter { + + private final static int DEFAULT_MERGE_NUMBER = 3750; + + private int accumTick; + private final int mergeField; + private IFrame lastFrame; + private IFrameTupleAccessor lastAccessor; + private int lastTupleTick; + private boolean updatedFrame; + private boolean updatedRange; + + /** + * @param ctx + * @param sampleFields + * @param sampleBasis + * @param comparators + * @param inRecordDesc + * @param outRecordDesc + * @param writer + * @param outputPartial + * @throws HyracksDataException + */ + public MergeOrderedSampleWriter(IHyracksTaskContext ctx, int[] sampleFields, int sampleBasis, + IBinaryComparator[] comparators, RecordDescriptor inRecordDesc, RecordDescriptor outRecordDesc, + IFrameWriter writer, boolean outputPartial) throws HyracksDataException { + super(ctx, sampleFields, sampleBasis, comparators, inRecordDesc, outRecordDesc, writer, outputPartial); + this.accumTick = 0; + this.mergeField = inRecordDesc.getFieldCount() - 1; + this.aggregator = new IntSumFieldAggregatorFactory(mergeField, true).createAggregator(ctx, inRecordDesc, + outRecordDesc); + this.aggregateState = aggregator.createState(); + this.sampleBasis = DEFAULT_MERGE_NUMBER; + } + + /** + * @param ctx + * @param sampleFields + * @param sampleBasis + * @param comparators + * @param inRecordDesc + * @param outRecordDesc + * @param writer + * @throws HyracksDataException + */ + public MergeOrderedSampleWriter(IHyracksTaskContext ctx, int[] sampleFields, int sampleBasis, + IBinaryComparator[] comparators, RecordDescriptor inRecordDesc, RecordDescriptor outRecordDesc, + IFrameWriter writer) throws HyracksDataException { + super(ctx, sampleFields, sampleBasis, comparators, inRecordDesc, outRecordDesc, writer); + this.accumTick = 0; + this.mergeField = inRecordDesc.getFieldCount() - 1; + this.aggregator = new IntSumFieldAggregatorFactory(mergeField, true).createAggregator(ctx, inRecordDesc, + outRecordDesc); + this.aggregateState = aggregator.createState(); + this.sampleBasis = DEFAULT_MERGE_NUMBER; + lastFrame = new VSizeFrame(ctx); + lastAccessor = new FrameTupleAccessor(inRecordDesc); + lastAccessor.reset(lastFrame.getBuffer()); + this.lastTupleTick = 0; + this.updatedRange = false; + } + + @Override + public void nextFrame(ByteBuffer buffer) throws HyracksDataException { + inFrameAccessor.reset(buffer); + int nTuples = inFrameAccessor.getTupleCount(); + for (int i = 0; i < nTuples; i++) { + accumTick += IntegerPointable.getInteger( + inFrameAccessor.getBuffer().array(), + inFrameAccessor.getTupleStartOffset(i) + inFrameAccessor.getFieldSlotsLength() + + inFrameAccessor.getFieldStartOffset(i, mergeField)); + if (isFirst) { + updatedFrame = true; + tupleBuilder.reset(); + for (int j = 0; j < sampleFields.length; j++) { + tupleBuilder.addField(inFrameAccessor, i, sampleFields[j]); + } + updateRangeAccessor(inFrameAccessor.getBuffer()); + lastTupleTick = i; + aggregator.init(inFrameAccessor, i, tupleBuilder.getDataOutput(), aggregateState); + isFirst = false; + } else { + if (i == 0) { + updatedFrame = true; + switchBinsIfRequired(copyFrameAccessor, copyFrameAccessor.getTupleCount() - 1, inFrameAccessor, i); + } else { + switchBinsIfRequired(inFrameAccessor, i - 1, inFrameAccessor, i); + } + } + } + copyFrame.ensureFrameSize(buffer.capacity()); + FrameUtils.copyAndFlip(buffer, copyFrame.getBuffer()); + copyFrameAccessor.reset(copyFrame.getBuffer()); + } + + private void updateRangeAccessor(ByteBuffer buf) throws HyracksDataException { + updatedRange = false; + if (updatedFrame) { + lastFrame.ensureFrameSize(buf.capacity()); + FrameUtils.copyAndFlip(buf, lastFrame.getBuffer()); + lastAccessor.reset(lastFrame.getBuffer()); + updatedFrame = false; + } + } + + @Override + protected void switchBinsIfRequired(IFrameTupleAccessor prevTA, int prevIdx, IFrameTupleAccessor currTA, int currIdx) + throws HyracksDataException { + if (!aggregatingWithBalanceGuaranteed(prevTA, prevIdx, currTA, currIdx) && accumTick > sampleBasis) { + if (updatedRange) { + writeOutput(lastAccessor, lastTupleTick); + updateRangeAccessor(currTA.getBuffer()); + lastTupleTick = currIdx; + accumTick = IntegerPointable.getInteger(currTA.getBuffer().array(), currTA.getTupleStartOffset(currIdx) + + currTA.getFieldSlotsLength() + currTA.getFieldStartOffset(currIdx, mergeField)); + tupleBuilder.reset(); + for (int i = 0; i < sampleFields.length; i++) { + tupleBuilder.addField(currTA, currIdx, sampleFields[i]); + } + aggregator.init(currTA, currIdx, tupleBuilder.getDataOutput(), aggregateState); + } else { + updatedRange = true; + aggregator.aggregate(currTA, currIdx, null, 0, aggregateState); + } + } else { + aggregator.aggregate(currTA, currIdx, null, 0, aggregateState); + } + } + + @Override + public void close() throws HyracksDataException { + if (!isFailed && !isFirst) { + assert (lastAccessor.getTupleCount() > 0); + writeOutput(lastAccessor, lastTupleTick); + } + super.close(); + } +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/OrderedSamplingWriter.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/OrderedSamplingWriter.java new file mode 100644 index 00000000000..aa3ba4a5968 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/OrderedSamplingWriter.java @@ -0,0 +1,180 @@ +/* + * 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.hyracks.dataflow.std.parallel.base; + +import java.nio.ByteBuffer; +import java.util.logging.Logger; + +import org.apache.hyracks.api.comm.IFrame; +import org.apache.hyracks.api.comm.IFrameTupleAccessor; +import org.apache.hyracks.api.comm.IFrameWriter; +import org.apache.hyracks.api.comm.VSizeFrame; +import org.apache.hyracks.api.context.IHyracksTaskContext; +import org.apache.hyracks.api.dataflow.value.IBinaryComparator; +import org.apache.hyracks.api.dataflow.value.RecordDescriptor; +import org.apache.hyracks.api.exceptions.HyracksDataException; +import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor; +import org.apache.hyracks.dataflow.common.comm.util.FrameUtils; + +/** + * @author michael + */ +public class OrderedSamplingWriter extends AbstractSamplingWriter { + private static final Logger LOGGER = Logger.getLogger(OrderedSamplingWriter.class.getName()); + + private final static int DEFAULT_TICKS_NUMBER = 501; + private final static int DEFAULT_TUPLE_NUMBER = 500; + private final int sampleTick; + private IFrame lastFrame; + private IFrameTupleAccessor lastAccessor; + private int lastTupleTick; + private int localTicks = 0; + private boolean updatedFrame; + + /** + * @param ctx + * @param sampleFields + * @param sampleBasis + * @param comparators + * @param inRecordDesc + * @param outRecordDesc + * @param writer + * @param outputPartial + * @throws HyracksDataException + */ + public OrderedSamplingWriter(IHyracksTaskContext ctx, int[] sampleFields, int sampleBasis, + IBinaryComparator[] comparators, RecordDescriptor inRecordDesc, RecordDescriptor outRecordDesc, + IFrameWriter writer, boolean outputPartial) throws HyracksDataException { + super(ctx, sampleFields, sampleBasis, comparators, inRecordDesc, outRecordDesc, writer, outputPartial); + this.sampleTick = DEFAULT_TICKS_NUMBER/*DEFAULT_TUPLE_NUMBER / this.sampleBasis + 1*/; + lastFrame = new VSizeFrame(ctx); + lastAccessor = new FrameTupleAccessor(inRecordDesc); + lastAccessor.reset(lastFrame.getBuffer()); + this.lastTupleTick = 0; + } + + /** + * @param ctx + * @param sampleFields + * @param sampleBasis + * @param comparators + * @param inRecordDesc + * @param outRecordDesc + * @param writer + * @throws HyracksDataException + */ + public OrderedSamplingWriter(IHyracksTaskContext ctx, int[] sampleFields, int sampleBasis, + IBinaryComparator[] comparators, RecordDescriptor inRecordDesc, RecordDescriptor outRecordDesc, + IFrameWriter writer) throws HyracksDataException { + super(ctx, sampleFields, sampleBasis, comparators, inRecordDesc, outRecordDesc, writer); + // TODO Auto-generated constructor stub + this.sampleTick = DEFAULT_TICKS_NUMBER/*DEFAULT_TUPLE_NUMBER / this.sampleBasis + 1*/; + lastFrame = new VSizeFrame(ctx); + lastAccessor = new FrameTupleAccessor(inRecordDesc); + lastAccessor.reset(lastFrame.getBuffer()); + this.lastTupleTick = 0; + } + + @Override + public void nextFrame(ByteBuffer buffer) throws HyracksDataException { + inFrameAccessor.reset(buffer); + int nTuples = inFrameAccessor.getTupleCount(); + for (int i = 0; i < nTuples; i++) { + localTicks++; + if (isFirst) { + updatedFrame = true; + tupleBuilder.reset(); + for (int j = 0; j < sampleFields.length; j++) { + tupleBuilder.addField(inFrameAccessor, i, sampleFields[j]); + } + cacheLastAccessor(inFrameAccessor.getBuffer()); + lastTupleTick = i; + aggregator.init(lastAccessor, lastTupleTick, tupleBuilder.getDataOutput(), aggregateState); + isFirst = false; + } else { + // each frame need to be at least sampled once. + if (i == 0) + updatedFrame = true; + switchBinsIfRequired(lastAccessor, lastTupleTick, inFrameAccessor, i); + } + } + } + + private void cacheLastAccessor(ByteBuffer buf) throws HyracksDataException { + if (updatedFrame) { + lastFrame.ensureFrameSize(buf.capacity()); + FrameUtils.copyAndFlip(buf, lastFrame.getBuffer()); + lastAccessor.reset(lastFrame.getBuffer()); + updatedFrame = false; + } + } + + @Override + protected boolean aggregatingWithBalanceGuaranteed(IFrameTupleAccessor prevTupleAccessor, int prevTupleIndex, + IFrameTupleAccessor currTupleAccessor, int currTupleIndex) throws HyracksDataException { + for (int i = 0; i < comparators.length; i++) { + int fIdx = sampleFields[i]; + int s1 = prevTupleAccessor.getAbsoluteFieldStartOffset(prevTupleIndex, fIdx); + int l1 = prevTupleAccessor.getFieldLength(prevTupleIndex, fIdx); + int s2 = currTupleAccessor.getAbsoluteFieldStartOffset(currTupleIndex, fIdx); + int l2 = currTupleAccessor.getFieldLength(currTupleIndex, fIdx); + if (0 != comparators[i].compare(prevTupleAccessor.getBuffer().array(), s1, l1, currTupleAccessor + .getBuffer().array(), s2, l2)) + return false; + } + return true; + } + + @Override + protected void switchBinsIfRequired(IFrameTupleAccessor prevTupleAccessor, int prevTupleIndex, + IFrameTupleAccessor currTupleAccessor, int currTupleIndex) throws HyracksDataException { + if (localTicks >= sampleTick) { + if (!aggregatingWithBalanceGuaranteed(lastAccessor, lastTupleTick, currTupleAccessor, currTupleIndex)) { + writeOutput(lastAccessor, lastTupleTick); + cacheLastAccessor(currTupleAccessor.getBuffer()); + lastTupleTick = currTupleIndex; + tupleBuilder.reset(); + for (int i = 0; i < sampleFields.length; i++) { + tupleBuilder.addField(currTupleAccessor, currTupleIndex, sampleFields[i]); + } + aggregator.init(currTupleAccessor, currTupleIndex, tupleBuilder.getDataOutput(), aggregateState); + localTicks = 1; + } else { + try { + aggregator.aggregate(lastAccessor, lastTupleTick, null, 0, aggregateState); + } catch (Exception e) { + LOGGER.info("Sampling error: " + tupleBuilder.getDataOutput().getClass().getName()); + throw new HyracksDataException("Failed to sample the immediate bins"); + } + } + } else { + aggregator.aggregate(lastAccessor, lastTupleTick, null, 0, aggregateState); + } + } + + @Override + public void close() throws HyracksDataException { + if (!isFailed && !isFirst) { + assert (lastAccessor.getTupleCount() > 0); + writeOutput(lastAccessor, lastTupleTick); + } + super.close(); + } +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/ParallelRangeMapTaskState.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/ParallelRangeMapTaskState.java new file mode 100644 index 00000000000..14857ea15cf --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/ParallelRangeMapTaskState.java @@ -0,0 +1,173 @@ +/* + * 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.hyracks.dataflow.std.parallel.base; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.nio.ByteBuffer; + +import org.apache.hyracks.api.comm.IFrame; +import org.apache.hyracks.api.comm.IFrameTupleAccessor; +import org.apache.hyracks.api.comm.IFrameWriter; +import org.apache.hyracks.api.comm.VSizeFrame; +import org.apache.hyracks.api.context.IHyracksTaskContext; +import org.apache.hyracks.api.dataflow.state.IStateObject; +import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer; +import org.apache.hyracks.api.dataflow.value.RecordDescriptor; +import org.apache.hyracks.api.exceptions.HyracksDataException; +import org.apache.hyracks.api.io.FileReference; +import org.apache.hyracks.api.job.JobId; +import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor; +import org.apache.hyracks.dataflow.common.data.partition.range.IRangeMap; +import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap; +import org.apache.hyracks.dataflow.common.io.RunFileReader; +import org.apache.hyracks.dataflow.common.io.RunFileWriter; + +/** + * @author michael + */ +public class ParallelRangeMapTaskState implements IStateObject/*extends AbstractStateObject*/{ + private IHyracksTaskContext ctx; + private RunFileWriter out; + private final RecordDescriptor keyDesc; + + public ParallelRangeMapTaskState(/*JobId jobId, Object taskId,*/RecordDescriptor keyDesc) { + // super(jobId, taskId); + + @SuppressWarnings("rawtypes") + ISerializerDeserializer[] schema = new ISerializerDeserializer[keyDesc.getFieldCount()]; + for (int i = 0; i < keyDesc.getFieldCount(); i++) { + schema[i] = keyDesc.getFields()[i]; + } + this.keyDesc = new RecordDescriptor(schema); + } + + @Override + public void toBytes(DataOutput out) throws IOException { + // TODO Auto-generated method stub + + } + + @Override + public void fromBytes(DataInput in) throws IOException { + // TODO Auto-generated method stub + } + + public void open(IHyracksTaskContext context) throws HyracksDataException { + this.ctx = context; + FileReference file = context.getJobletContext().createManagedWorkspaceFile( + ParallelRangeMapTaskState.class.getSimpleName()); + out = new RunFileWriter(file, context.getIOManager()); + out.open(); + } + + public void close() throws HyracksDataException { + out.close(); + } + + public void appendFrame(ByteBuffer buf) throws HyracksDataException { + out.nextFrame(buf); + } + + public void writeOut(IFrameWriter writer, IFrame frame) throws HyracksDataException { + RunFileReader in = out.createReader(); + writer.open(); + try { + in.open(); + while (in.nextFrame(frame)) { + writer.nextFrame(frame.getBuffer()); + } + in.close(); + } catch (Exception e) { + writer.fail(); + throw new HyracksDataException(e); + } finally { + writer.close(); + } + } + + public void deleteFile() { + out.getFileReference().delete(); + } + + public IRangeMap getRangeMap() throws HyracksDataException { + IFrame frame = new VSizeFrame(ctx); + // out.open(); + RunFileReader in = out.createReader(); + IFrameTupleAccessor inAccessor = new FrameTupleAccessor(keyDesc); + int byteLen = 0; + int tupleCount = 0; + in.open(); + while (in.nextFrame(frame)) { + inAccessor.reset(frame.getBuffer()); + int nTuple = inAccessor.getTupleCount(); + tupleCount += nTuple; + for (int i = 0; i < nTuple; i++) { + for (int j = 0; j < keyDesc.getFieldCount() - 1; j++) { + byteLen += inAccessor.getFieldLength(i, j); + } + } + } + in.close(); + in.open(); + byte[] byteRange = new byte[byteLen]; + int[] offRange = new int[tupleCount]; + int byteCur = 0; + while (in.nextFrame(frame)) { + inAccessor.reset(frame.getBuffer()); + int nTuple = inAccessor.getTupleCount(); + for (int i = 0; i < nTuple; i++) { + offRange[i] = byteCur; + for (int j = 0; j < keyDesc.getFieldCount() - 1; j++) { + int tStart = inAccessor.getTupleStartOffset(i); + int fStart = inAccessor.getFieldStartOffset(i, j); + int fEnd = inAccessor.getFieldEndOffset(i, j); + System.arraycopy(inAccessor.getBuffer().array(), + tStart + inAccessor.getFieldSlotsLength() + fStart, byteRange, byteCur, fEnd - fStart); + byteCur += (fEnd - fStart); + } + } + } + in.close(); + // out.close(); + IRangeMap rangeMap = new RangeMap(keyDesc.getFieldCount() - 1, byteRange, offRange); + return rangeMap; + } + + @Override + public JobId getJobId() { + // TODO Auto-generated method stub + return null; + } + + @Override + public Object getId() { + // TODO Auto-generated method stub + return null; + } + + @Override + public long getMemoryOccupancy() { + // TODO Auto-generated method stub + return 0; + } + +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/QuantileSamplingWriter.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/QuantileSamplingWriter.java new file mode 100644 index 00000000000..cc24f68a258 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/base/QuantileSamplingWriter.java @@ -0,0 +1,99 @@ +/* + * 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.hyracks.dataflow.std.parallel.base; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.Map.Entry; + +import org.apache.hyracks.api.comm.IFrameWriter; +import org.apache.hyracks.api.context.IHyracksTaskContext; +import org.apache.hyracks.api.dataflow.value.IBinaryComparator; +import org.apache.hyracks.api.dataflow.value.RecordDescriptor; +import org.apache.hyracks.api.exceptions.HyracksDataException; +import org.apache.hyracks.data.std.util.GrowableArray; +import org.apache.hyracks.dataflow.std.parallel.IHistogram; +import org.apache.hyracks.dataflow.std.parallel.histogram.structures.AbstractStreamingHistogram; + +/** + * @author michael + */ +public class QuantileSamplingWriter extends AbstractSamplingWriter { + private static int processed = 0; + private final IHyracksTaskContext ctx; + private final int[] sampleFields; + private final int sampleBasis; + private final IBinaryComparator[] comparators; + private final RecordDescriptor inDesc; + private final RecordDescriptor outDesc; + private final boolean outputPartial; + + private static final int DEFAULT_ELASTIC = 10; + private static final double DEFAULT_MU = 1.7; + + private final IHistogram merging; + private final IHistogram current; + + public QuantileSamplingWriter(IHyracksTaskContext ctx, int[] sampleFields, int sampleBasis, + IBinaryComparator[] comparators, RecordDescriptor inRecordDesc, RecordDescriptor outRecordDesc, + IFrameWriter writer, boolean outputPartial) throws HyracksDataException { + super(ctx, sampleFields, sampleBasis, comparators, inRecordDesc, outRecordDesc, writer); + this.ctx = ctx; + this.sampleFields = sampleFields; + this.sampleBasis = sampleBasis; + this.comparators = comparators; + this.inDesc = inRecordDesc; + this.outDesc = outRecordDesc; + this.outputPartial = outputPartial; + this.merging = new AbstractStreamingHistogram(comparators, sampleBasis, DEFAULT_ELASTIC, DEFAULT_MU); + this.current = new AbstractStreamingHistogram(comparators, sampleBasis, DEFAULT_ELASTIC, DEFAULT_MU); + } + + public QuantileSamplingWriter(IHyracksTaskContext ctx, int[] sampleFields, int sampleBasis, + IBinaryComparator[] comparators, RecordDescriptor inRecordDesc, RecordDescriptor outRecordDesc, + IFrameWriter writer) throws HyracksDataException { + this(ctx, sampleFields, sampleBasis, comparators, inRecordDesc, outRecordDesc, writer, false); + } + + @Override + public void open() throws HyracksDataException { + } + + @Override + public void nextFrame(ByteBuffer buffer) throws HyracksDataException { + } + + @Override + public void fail() throws HyracksDataException { + isFailed = true; + appenderWrapper.fail(); + } + + @Override + public void close() throws HyracksDataException { + if (!isFailed && !isFirst) { + appenderWrapper.flush(); + } + aggregator.close(); + aggregateState.close(); + appenderWrapper.close(); + } +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/AbstractSampleOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/AbstractSampleOperatorDescriptor.java new file mode 100644 index 00000000000..7cd90fb6821 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/AbstractSampleOperatorDescriptor.java @@ -0,0 +1,202 @@ +/* + * 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.hyracks.dataflow.std.parallel.histogram; + +import org.apache.hyracks.api.comm.VSizeFrame; +import org.apache.hyracks.api.context.IHyracksTaskContext; +import org.apache.hyracks.api.dataflow.ActivityId; +import org.apache.hyracks.api.dataflow.IActivityGraphBuilder; +import org.apache.hyracks.api.dataflow.IOperatorNodePushable; +import org.apache.hyracks.api.dataflow.TaskId; +import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory; +import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider; +import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer; +import org.apache.hyracks.api.dataflow.value.RecordDescriptor; +import org.apache.hyracks.api.exceptions.HyracksDataException; +import org.apache.hyracks.api.job.IOperatorDescriptorRegistry; +import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer; +import org.apache.hyracks.dataflow.std.base.AbstractActivityNode; +import org.apache.hyracks.dataflow.std.base.AbstractOperatorDescriptor; +import org.apache.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable; +import org.apache.hyracks.dataflow.std.parallel.HistogramAlgorithm; +import org.apache.hyracks.dataflow.std.parallel.base.MaterializingSampleTaskState; + +/** + * @author michael + * @Comments: This Descriptor can be seen as the local sampling method for the uniform, random and wavelet based samplers plus the + * materialization for both the sample and the input dataset, the merge part of the sampler is given in the following + * AbstractSampleMergeOperatorDescriptor after a MToOneExchange. + */ +public abstract class AbstractSampleOperatorDescriptor extends AbstractOperatorDescriptor { + + // private static final Logger LOGGER = Logger.getLogger(AbstractSampleOperatorDescriptor.class.getName()); + + private static final long serialVersionUID = 1L; + + private final static int LOCAL_SAMPLING_FACTOR = 2; + + private final static int MATER_SAMPLER_ACTIVITY_ID = 0; + private final static int MATER_READER_ACTIVITY_ID = 1; + + // sampleMaterializationFlags numbers the samples plus the count of the materialization count + private final HistogramAlgorithm alg; + private boolean[] sampleMaterializationFlags; + private boolean requiresMaterialization; + private int numberOfNonMaterializedOutputs = 0; + private int numberOfActiveMaterializeReaders = 0; + + private final int[] sampleFields; + private final int sampleBasis; + private IBinaryComparatorFactory[] comparatorFactories; + + private RecordDescriptor outDesc; + private RecordDescriptor inDesc; + + public AbstractSampleOperatorDescriptor(IOperatorDescriptorRegistry spec, int frameLimit, int[] sampleFields, + int sampleBasis, RecordDescriptor rDesc, IBinaryComparatorFactory[] compFactories, HistogramAlgorithm alg, + int outputArity) { + this(spec, frameLimit, sampleFields, sampleBasis, rDesc, compFactories, alg, outputArity, + new boolean[outputArity]); + } + + public AbstractSampleOperatorDescriptor(IOperatorDescriptorRegistry spec, int frameLimit, int[] sampleFields, + int sampleBasis, RecordDescriptor rDesc, IBinaryComparatorFactory[] compFactories, HistogramAlgorithm alg, + int outputArity, boolean[] outputMaterializationFlags) { + super(spec, 1, outputArity + 1); + // Column 0 for sampling column(s) for feeding the range of those joinable operators. length for sampleKey point, 1 for count + @SuppressWarnings("rawtypes") + ISerializerDeserializer[] outSchema = new ISerializerDeserializer[sampleFields.length + 1]; + for (int i = 0; i < sampleFields.length; i++) { + outSchema[i] = rDesc.getFields()[sampleFields[i]]; + } + outSchema[outSchema.length - 1] = IntegerSerializerDeserializer.INSTANCE; + this.outDesc = new RecordDescriptor(outSchema); + this.inDesc = rDesc; + if (outputArity <= 0) { + recordDescriptors[0] = outDesc; + } else { + recordDescriptors[0] = outDesc; + for (int i = 1; i <= outputArity; i++) { + recordDescriptors[i] = rDesc; + } + } + // this.sampleMaterializationFlags = outputMaterializationFlags; + this.sampleMaterializationFlags = new boolean[outputMaterializationFlags.length + 1]; + this.sampleMaterializationFlags[0] = false; + System.arraycopy(outputMaterializationFlags, 0, this.sampleMaterializationFlags, 1, outputArity); + this.sampleFields = sampleFields; + // sampleBasis is desired by the UC numbers * GLOBAL_FACTOR * LOCAL_SMPLEING_FACTOR, while the LOCAL is merged in total. + // The actual ranges need to merge is MOST LIKELY sampleBasis * NC numbers and will be detailed by sampling algorithms. + this.sampleBasis = sampleBasis * LOCAL_SAMPLING_FACTOR; + this.comparatorFactories = compFactories; + this.requiresMaterialization = false; + for (boolean flag : this.sampleMaterializationFlags) { + if (flag) { + this.requiresMaterialization = true; + break; + } + } + if (null == alg) + this.alg = HistogramAlgorithm.ORDERED_HISTOGRAM; + else + this.alg = alg; + } + + @Override + public void contributeActivities(IActivityGraphBuilder builder) { + MaterializingSamplerActivityNode msa = new MaterializingSamplerActivityNode(new ActivityId(odId, + MATER_SAMPLER_ACTIVITY_ID)); + builder.addActivity(this, msa); + builder.addSourceEdge(0, msa, 0); + int outputIndex = 0; + + for (int i = 0; i < outputArity; i++) { + if (!sampleMaterializationFlags[i]) { + builder.addTargetEdge(i, msa, outputIndex); + outputIndex++; + } + } + + numberOfNonMaterializedOutputs = outputIndex; + int activityId = MATER_READER_ACTIVITY_ID; + for (int i = 0; i < outputArity; i++) { + if (sampleMaterializationFlags[i]) { + MaterializedSampleReaderActivityNode msra = new MaterializedSampleReaderActivityNode(new ActivityId( + odId, activityId)); + builder.addActivity(this, msra); + // builder.addSourceEdge(1, msra, 0); + builder.addTargetEdge(i, msra, 0); + builder.addBlockingEdge(msa, msra); + activityId++; + outputIndex++; + numberOfActiveMaterializeReaders++; + } + } + } + + private class MaterializingSamplerActivityNode extends AbstractActivityNode { + private static final long serialVersionUID = 1L; + + public MaterializingSamplerActivityNode(ActivityId id) { + super(id); + } + + @Override + public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, + IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) + throws HyracksDataException { + return new MaterializingSampleOperatorNodePushable(ctx, new TaskId(getActivityId(), partition), + sampleFields, sampleBasis, comparatorFactories, alg, inDesc, outDesc, + numberOfNonMaterializedOutputs, requiresMaterialization, partition); + } + } + + private class MaterializedSampleReaderActivityNode extends AbstractActivityNode { + private static final long serialVersionUID = 1L; + + public MaterializedSampleReaderActivityNode(ActivityId id) { + super(id); + // TODO Auto-generated constructor stub + } + + @Override + public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, + IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) + throws HyracksDataException { + return new AbstractUnaryOutputSourceOperatorNodePushable() { + @Override + public void initialize() throws HyracksDataException { + MaterializingSampleTaskState state = (MaterializingSampleTaskState) ctx.getStateObject(new TaskId( + new ActivityId(getOperatorId(), MATER_SAMPLER_ACTIVITY_ID), partition)); + state.writeOut(writer, new VSizeFrame(ctx)); + } + + @Override + public void deinitialize() throws HyracksDataException { + numberOfActiveMaterializeReaders--; + MaterializingSampleTaskState state = (MaterializingSampleTaskState) ctx.getStateObject(new TaskId( + new ActivityId(getOperatorId(), MATER_SAMPLER_ACTIVITY_ID), partition)); + if (numberOfActiveMaterializeReaders == 0) + state.deleteFile(); + } + }; + } + } +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/BlockingForwardOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/BlockingForwardOperatorDescriptor.java new file mode 100644 index 00000000000..7b01be8263c --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/BlockingForwardOperatorDescriptor.java @@ -0,0 +1,127 @@ +/* + * 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.hyracks.dataflow.std.parallel.histogram; + +import java.nio.ByteBuffer; + +import org.apache.hyracks.api.comm.VSizeFrame; +import org.apache.hyracks.api.context.IHyracksTaskContext; +import org.apache.hyracks.api.dataflow.ActivityId; +import org.apache.hyracks.api.dataflow.IActivityGraphBuilder; +import org.apache.hyracks.api.dataflow.IOperatorNodePushable; +import org.apache.hyracks.api.dataflow.TaskId; +import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory; +import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider; +import org.apache.hyracks.api.dataflow.value.RecordDescriptor; +import org.apache.hyracks.api.exceptions.HyracksDataException; +import org.apache.hyracks.api.job.IOperatorDescriptorRegistry; +import org.apache.hyracks.dataflow.std.base.AbstractActivityNode; +import org.apache.hyracks.dataflow.std.base.AbstractOperatorDescriptor; +import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable; +import org.apache.hyracks.dataflow.std.parallel.base.MaterializingSampleTaskState; + +/** + * @author michael + */ +public class BlockingForwardOperatorDescriptor extends AbstractOperatorDescriptor { + + private static final long serialVersionUID = 1L; + + private final RecordDescriptor sampleDesc; + + private final static int SAMPLED_RANGE_ACTIVITY_ID = 0; + private final static int FRAME_FORWARD_ACTIVITY_ID = 1; + + public BlockingForwardOperatorDescriptor(IOperatorDescriptorRegistry spec, int frameLimit, int[] sampleFields, + RecordDescriptor inSampleDesc, RecordDescriptor inDataDesc, IBinaryComparatorFactory[] compFactories) { + super(spec, 2, 1); + this.sampleDesc = inSampleDesc; + this.recordDescriptors[0] = inDataDesc; + } + + /* (non-Javadoc) + * @see org.apache.hyracks.api.dataflow.IOperatorDescriptor#contributeActivities(org.apache.hyracks.api.dataflow.IActivityGraphBuilder) + */ + @Override + public void contributeActivities(IActivityGraphBuilder builder) { + // Currently, the streaming materialization is enforced during the initial phase of sampling. + RangeMapReaderActivityNode sra = new RangeMapReaderActivityNode(new ActivityId(odId, SAMPLED_RANGE_ACTIVITY_ID)); + builder.addActivity(this, sra); + builder.addSourceEdge(0, sra, 0); + BlockingForwardActivityNode bfa = new BlockingForwardActivityNode(new ActivityId(odId, + FRAME_FORWARD_ACTIVITY_ID)); + builder.addActivity(this, bfa); + builder.addSourceEdge(1, bfa, 0); + builder.addBlockingEdge(sra, bfa); + } + + private final class RangeMapReaderActivityNode extends AbstractActivityNode { + private static final long serialVersionUID = 1L; + + public RangeMapReaderActivityNode(ActivityId id) { + super(id); + } + + @Override + public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, + IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) { + return new ForwardOperatorNodePushable(ctx, sampleDesc, partition); + } + } + + private final class BlockingForwardActivityNode extends AbstractActivityNode { + private static final long serialVersionUID = 1L; + + public BlockingForwardActivityNode(ActivityId id) { + super(id); + } + + @Override + public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, + IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) { + return new AbstractUnaryInputUnaryOutputOperatorNodePushable() { + private MaterializingSampleTaskState dataState; + + @Override + public void open() throws HyracksDataException { + dataState = new MaterializingSampleTaskState(ctx.getJobletContext().getJobId(), new TaskId( + getActivityId(), partition)); + dataState.open(ctx); + } + + @Override + public void nextFrame(ByteBuffer buffer) throws HyracksDataException { + dataState.appendFrame(buffer); + } + + @Override + public void fail() throws HyracksDataException { + } + + @Override + public void close() throws HyracksDataException { + dataState.close(); + dataState.writeOut(writer, new VSizeFrame(ctx)); + } + + }; + } + } +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/ForwardOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/ForwardOperatorNodePushable.java new file mode 100644 index 00000000000..cfbf2178469 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/ForwardOperatorNodePushable.java @@ -0,0 +1,67 @@ +/* + * 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.hyracks.dataflow.std.parallel.histogram; + +import java.nio.ByteBuffer; + +import org.apache.hyracks.api.context.IHyracksTaskContext; +import org.apache.hyracks.api.dataflow.value.RecordDescriptor; +import org.apache.hyracks.api.exceptions.HyracksDataException; +import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable; +import org.apache.hyracks.dataflow.std.parallel.base.ParallelRangeMapTaskState; + +/** + * @author michael + */ +public class ForwardOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable { + + private final IHyracksTaskContext ctx; + private final RecordDescriptor sampleDesc; + private ParallelRangeMapTaskState sampleState; + private int partition; + + public ForwardOperatorNodePushable(IHyracksTaskContext ctx, RecordDescriptor sampleDesc, int partition) { + this.ctx = ctx; + this.sampleDesc = sampleDesc; + this.partition = partition; + } + + @Override + public void open() throws HyracksDataException { + ctx.setGlobalState(partition, new ParallelRangeMapTaskState(sampleDesc)); + sampleState = (ParallelRangeMapTaskState) ctx.getGlobalState(partition); + sampleState.open(ctx); + } + + @Override + public void nextFrame(ByteBuffer buffer) throws HyracksDataException { + sampleState.appendFrame(buffer); + } + + @Override + public void fail() throws HyracksDataException { + } + + @Override + public void close() throws HyracksDataException { + sampleState.close(); + } + +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/MaterializingForwardOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/MaterializingForwardOperatorDescriptor.java new file mode 100644 index 00000000000..1144314a686 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/MaterializingForwardOperatorDescriptor.java @@ -0,0 +1,167 @@ +/* + * 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.hyracks.dataflow.std.parallel.histogram; + +import java.nio.ByteBuffer; + +import org.apache.hyracks.api.comm.VSizeFrame; +import org.apache.hyracks.api.context.IHyracksTaskContext; +import org.apache.hyracks.api.dataflow.ActivityId; +import org.apache.hyracks.api.dataflow.IActivityGraphBuilder; +import org.apache.hyracks.api.dataflow.IOperatorNodePushable; +import org.apache.hyracks.api.dataflow.TaskId; +import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory; +import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider; +import org.apache.hyracks.api.dataflow.value.RecordDescriptor; +import org.apache.hyracks.api.exceptions.HyracksDataException; +import org.apache.hyracks.api.job.IOperatorDescriptorRegistry; +import org.apache.hyracks.dataflow.std.base.AbstractActivityNode; +import org.apache.hyracks.dataflow.std.base.AbstractOperatorDescriptor; +import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable; +import org.apache.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable; +import org.apache.hyracks.dataflow.std.parallel.base.MaterializingSampleTaskState; + +/** + * @author michael + */ +public class MaterializingForwardOperatorDescriptor extends AbstractOperatorDescriptor { + + private static final long serialVersionUID = 1L; + + private final RecordDescriptor sampleDesc; + + private final static int SAMPLED_RANGE_ACTIVITY_ID = 0; + private final static int MATER_FORWARD_ACTIVITY_ID = 1; + private final static int MATER_READER_ACTIVITY_ID = 2; + + /** + * @param spec + * @param inputArity + * @param outputArity + */ + public MaterializingForwardOperatorDescriptor(IOperatorDescriptorRegistry spec, int frameLimit, int[] sampleFields, + RecordDescriptor inSampleDesc, RecordDescriptor inDataDesc, IBinaryComparatorFactory[] compFactories) { + super(spec, 2, 1); + this.sampleDesc = inSampleDesc; + this.recordDescriptors[0] = inDataDesc; + } + + @Override + public void contributeActivities(IActivityGraphBuilder builder) { + // Currently, the streaming materialization is enforced during the initial phase of sampling. + SampledRangeActivityNode sra = new SampledRangeActivityNode(new ActivityId(odId, SAMPLED_RANGE_ACTIVITY_ID)); + builder.addActivity(this, sra); + builder.addSourceEdge(0, sra, 0); + MaterializingForwardActivityNode mfa = new MaterializingForwardActivityNode(new ActivityId(odId, + MATER_FORWARD_ACTIVITY_ID)); + builder.addActivity(this, mfa); + builder.addSourceEdge(1, mfa, 0); + builder.addBlockingEdge(sra, mfa); + // builder.addTargetEdge(0, mfa, 0); + MaterializedReaderActivityNode mra = new MaterializedReaderActivityNode(new ActivityId(odId, + MATER_READER_ACTIVITY_ID)); + builder.addActivity(this, mra); + builder.addBlockingEdge(mfa, mra); + builder.addTargetEdge(0, mra, 0); + } + + private final class SampledRangeActivityNode extends AbstractActivityNode { + private static final long serialVersionUID = 1L; + + public SampledRangeActivityNode(ActivityId id) { + super(id); + } + + @Override + public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, + IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) { + return new ForwardOperatorNodePushable(ctx, sampleDesc, partition); + } + } + + private final class MaterializingForwardActivityNode extends AbstractActivityNode { + private static final long serialVersionUID = 1L; + + public MaterializingForwardActivityNode(ActivityId id) { + super(id); + } + + @Override + public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, + IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) { + return new AbstractUnaryInputUnaryOutputOperatorNodePushable() { + private MaterializingSampleTaskState dataState; + + @Override + public void open() throws HyracksDataException { + dataState = new MaterializingSampleTaskState(ctx.getJobletContext().getJobId(), new TaskId( + getActivityId(), partition)); + dataState.open(ctx); + } + + @Override + public void nextFrame(ByteBuffer buffer) throws HyracksDataException { + dataState.appendFrame(buffer); + } + + @Override + public void fail() throws HyracksDataException { + } + + @Override + public void close() throws HyracksDataException { + dataState.close(); + // dataState.writeOut(writer, new VSizeFrame(ctx)); + ctx.setStateObject(dataState); + } + + }; + } + } + + private final class MaterializedReaderActivityNode extends AbstractActivityNode { + private static final long serialVersionUID = 1L; + + public MaterializedReaderActivityNode(ActivityId id) { + super(id); + } + + @Override + public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, + IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) { + + return new AbstractUnaryOutputSourceOperatorNodePushable() { + @Override + public void initialize() throws HyracksDataException { + MaterializingSampleTaskState state = (MaterializingSampleTaskState) ctx.getStateObject(new TaskId( + new ActivityId(getOperatorId(), MATER_FORWARD_ACTIVITY_ID), partition)); + state.writeOut(writer, new VSizeFrame(ctx)); + } + + @Override + public void deinitialize() throws HyracksDataException { + MaterializingSampleTaskState state = (MaterializingSampleTaskState) ctx.getStateObject(new TaskId( + new ActivityId(getOperatorId(), MATER_FORWARD_ACTIVITY_ID), partition)); + state.deleteFile(); + } + }; + } + } +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/MaterializingSampleOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/MaterializingSampleOperatorDescriptor.java new file mode 100644 index 00000000000..768a7c6cbf5 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/MaterializingSampleOperatorDescriptor.java @@ -0,0 +1,69 @@ +/* + * 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.hyracks.dataflow.std.parallel.histogram; + +import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory; +import org.apache.hyracks.api.dataflow.value.RecordDescriptor; +import org.apache.hyracks.api.job.IOperatorDescriptorRegistry; +import org.apache.hyracks.dataflow.std.parallel.HistogramAlgorithm; + +/** + * @author michael + */ +public class MaterializingSampleOperatorDescriptor extends AbstractSampleOperatorDescriptor { + private static final long serialVersionUID = 1L; + + /** + * @param spec + * @param frameLimit + * @param sampleFields + * @param sampleBasis + * @param rDesc + * @param compFactories + * @param alg + * @param outputArity + */ + public MaterializingSampleOperatorDescriptor(IOperatorDescriptorRegistry spec, int frameLimit, int[] sampleFields, + int sampleBasis, RecordDescriptor rDesc, IBinaryComparatorFactory[] compFactories, HistogramAlgorithm alg, + int outputArity) { + super(spec, frameLimit, sampleFields, sampleBasis, rDesc, compFactories, alg, outputArity); + // TODO Auto-generated constructor stub + } + + /** + * @param spec + * @param frameLimit + * @param sampleFields + * @param sampleBasis + * @param rDesc + * @param compFactories + * @param alg + * @param outputArity + * @param outputMaterializationFlags + */ + public MaterializingSampleOperatorDescriptor(IOperatorDescriptorRegistry spec, int frameLimit, int[] sampleFields, + int sampleBasis, RecordDescriptor rDesc, IBinaryComparatorFactory[] compFactories, HistogramAlgorithm alg, + int outputArity, boolean[] outputMaterializationFlags) { + super(spec, frameLimit, sampleFields, sampleBasis, rDesc, compFactories, alg, outputArity, + outputMaterializationFlags); + // TODO Auto-generated constructor stub + } + +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/MaterializingSampleOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/MaterializingSampleOperatorNodePushable.java new file mode 100644 index 00000000000..a472591a6f7 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/MaterializingSampleOperatorNodePushable.java @@ -0,0 +1,151 @@ +/* + * 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.hyracks.dataflow.std.parallel.histogram; + +import java.nio.ByteBuffer; + +import org.apache.hyracks.api.comm.IFrame; +import org.apache.hyracks.api.comm.IFrameWriter; +import org.apache.hyracks.api.comm.VSizeFrame; +import org.apache.hyracks.api.context.IHyracksTaskContext; +import org.apache.hyracks.api.dataflow.value.IBinaryComparator; +import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory; +import org.apache.hyracks.api.dataflow.value.RecordDescriptor; +import org.apache.hyracks.api.exceptions.HyracksDataException; +import org.apache.hyracks.api.io.FileReference; +import org.apache.hyracks.dataflow.common.comm.util.FrameUtils; +import org.apache.hyracks.dataflow.common.io.RunFileReader; +import org.apache.hyracks.dataflow.common.io.RunFileWriter; +import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputOperatorNodePushable; +import org.apache.hyracks.dataflow.std.parallel.HistogramAlgorithm; +import org.apache.hyracks.dataflow.std.parallel.base.AbstractSamplingWriter; +import org.apache.hyracks.dataflow.std.parallel.base.MaterializingSampleTaskState; +import org.apache.hyracks.dataflow.std.parallel.base.OrderedSamplingWriter; +import org.apache.hyracks.dataflow.std.parallel.base.QuantileSamplingWriter; + +/** + * @author michael + */ +public class MaterializingSampleOperatorNodePushable extends AbstractUnaryInputOperatorNodePushable { + + private AbstractSamplingWriter sw; + // private RunFileWriter swriter; + private final IHyracksTaskContext ctx; + private final int[] sampleFields; + private final int sampleBasis; + private final IBinaryComparatorFactory[] comparatorFactories; + private final HistogramAlgorithm alg; + private final RecordDescriptor inDesc; + private final RecordDescriptor outDesc; + private final boolean requiresMaterialization; + private final int partition; + private final int nNonMaterialization; + private final Object stateId; + + private MaterializingSampleTaskState state; + private final IFrameWriter[] writers; + + public MaterializingSampleOperatorNodePushable(IHyracksTaskContext ctx, Object stateId, int[] sampleFields, + int sampleBasis, IBinaryComparatorFactory[] comparatorFactories, HistogramAlgorithm alg, + RecordDescriptor inDesc, RecordDescriptor outDesc, int nNonMaterialization, + boolean requiresMaterialization, int partition) { + this.ctx = ctx; + this.stateId = stateId; + this.sampleFields = sampleFields; + this.sampleBasis = sampleBasis; + this.comparatorFactories = comparatorFactories; + this.alg = alg; + this.inDesc = inDesc; + this.outDesc = outDesc; + this.requiresMaterialization = requiresMaterialization; + this.partition = partition; + this.nNonMaterialization = nNonMaterialization; + this.writers = new IFrameWriter[nNonMaterialization]; + } + + @Override + public void open() throws HyracksDataException { + // sample first + IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length]; + for (int i = 0; i < comparatorFactories.length; ++i) { + comparators[i] = comparatorFactories[i].createBinaryComparator(); + } + // extract rangePartition and dynamically promote the partition param, get the RangeConnector and change the rangeMap. + if (requiresMaterialization) { + state = new MaterializingSampleTaskState(ctx.getJobletContext().getJobId(), stateId); + state.open(ctx); + } + for (int i = 0; i < nNonMaterialization; i++) { + writers[i].open(); + } + + // Here for single input sampler only and will be merged for multiway sampling in the merge part. + switch (alg) { + case ORDERED_HISTOGRAM: + sw = new OrderedSamplingWriter(ctx, sampleFields, sampleBasis, comparators, inDesc, outDesc, + writers[0], false); + break; + case UNIFORM_HISTOGRAM: + sw = new QuantileSamplingWriter(ctx, sampleFields, sampleBasis, comparators, inDesc, outDesc, + writers[0], false); + break; + } + sw.open(); + } + + @Override + public void nextFrame(ByteBuffer buffer) throws HyracksDataException { + sw.nextFrame(buffer); + buffer.rewind(); + // if (requiresMaterialization) { + state.appendFrame(buffer); + // } + for (int i = 1; i < nNonMaterialization; i++) { + FrameUtils.flushFrame(buffer, writers[i]); + } + } + + @Override + public void fail() throws HyracksDataException { + sw.fail(); + for (int i = 0; i < nNonMaterialization; i++) { + writers[i].fail(); + } + } + + @Override + public void close() throws HyracksDataException { + if (requiresMaterialization) { + state.close(); + ctx.setStateObject(state); + } + sw.close(); + for (int i = 1; i < nNonMaterialization; i++) { + state.writeOut(writers[i], new VSizeFrame(ctx)); + writers[i].close(); + } + } + + @Override + public void setOutputFrameWriter(int index, IFrameWriter writer, RecordDescriptor recrdDesc) { + // this.writer = writer; + writers[index] = writer; + } +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/MergeSampleOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/MergeSampleOperatorDescriptor.java new file mode 100644 index 00000000000..e7643a2db1b --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/MergeSampleOperatorDescriptor.java @@ -0,0 +1,150 @@ +/* + * 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.hyracks.dataflow.std.parallel.histogram; + +import org.apache.hyracks.api.context.IHyracksTaskContext; +import org.apache.hyracks.api.dataflow.ActivityId; +import org.apache.hyracks.api.dataflow.IActivityGraphBuilder; +import org.apache.hyracks.api.dataflow.IOperatorNodePushable; +import org.apache.hyracks.api.dataflow.TaskId; +import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory; +import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory; +import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider; +import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer; +import org.apache.hyracks.api.dataflow.value.RecordDescriptor; +import org.apache.hyracks.api.exceptions.HyracksDataException; +import org.apache.hyracks.api.job.IOperatorDescriptorRegistry; +import org.apache.hyracks.dataflow.std.base.AbstractActivityNode; +import org.apache.hyracks.dataflow.std.base.AbstractOperatorDescriptor; +import org.apache.hyracks.dataflow.std.base.AbstractUnaryOutputSourceOperatorNodePushable; +import org.apache.hyracks.dataflow.std.parallel.HistogramAlgorithm; +import org.apache.hyracks.dataflow.std.sort.Algorithm; +import org.apache.hyracks.dataflow.std.sort.FrameSorterMergeSort; +import org.apache.hyracks.dataflow.std.sort.FrameSorterQuickSort; +import org.apache.hyracks.dataflow.std.sort.IFrameSorter; + +/** + * @author michael + */ +public class MergeSampleOperatorDescriptor extends AbstractOperatorDescriptor { + private static final long serialVersionUID = 1L; + + private final static int GLOBAL_MERGE_FACTOR = 2; + + private final static int MERGE_SAMPLER_ACTIVITY_ID = 0; + private final static int SAMPLE_READER_ACTIVITY_ID = 1; + + private final int frameLimit; + private final int outputLimit; + private final HistogramAlgorithm algorithm; + + private final int[] sampleFields; + private final int sampleBasis; + private IBinaryComparatorFactory[] comparatorFactories; + private INormalizedKeyComputerFactory firstKeyNormalizerFactory; + + private RecordDescriptor outDesc; + private RecordDescriptor inDesc; + boolean needMaterialization; + + /** + * @param spec + * @param inputArity + * @param outputArity + * @throws HyracksDataException + */ + public MergeSampleOperatorDescriptor(IOperatorDescriptorRegistry spec, int frameLimit, int[] sampleFields, + RecordDescriptor inDesc, int outputLimit, INormalizedKeyComputerFactory firstKeyNormalizerFactory, + IBinaryComparatorFactory[] compFactories, HistogramAlgorithm alg, boolean needMaterialization) + throws HyracksDataException { + super(spec, 1, 1); + this.frameLimit = frameLimit; + this.outputLimit = outputLimit; + this.algorithm = alg; + this.sampleBasis = GLOBAL_MERGE_FACTOR; + this.sampleFields = sampleFields; + this.comparatorFactories = compFactories; + this.firstKeyNormalizerFactory = firstKeyNormalizerFactory; + this.inDesc = inDesc; + this.needMaterialization = needMaterialization; + this.outDesc = inDesc; + this.recordDescriptors[0] = inDesc; + } + + @Override + public void contributeActivities(IActivityGraphBuilder builder) { + if (!needMaterialization) { + MergeSampleActivityNode msa = new MergeSampleActivityNode(new ActivityId(odId, MERGE_SAMPLER_ACTIVITY_ID)); + builder.addActivity(this, msa); + builder.addSourceEdge(0, msa, 0); + builder.addTargetEdge(0, msa, 0); + } else { + MergeSampleActivityNode msa = new MergeSampleActivityNode(new ActivityId(odId, MERGE_SAMPLER_ACTIVITY_ID)); + SampleReaderActivityNode sra = new SampleReaderActivityNode(new ActivityId(odId, SAMPLE_READER_ACTIVITY_ID)); + builder.addActivity(this, msa); + builder.addSourceEdge(0, msa, 0); + builder.addActivity(this, sra); + builder.addTargetEdge(0, sra, 0); + builder.addBlockingEdge(msa, sra); + } + } + + private class MergeSampleActivityNode extends AbstractActivityNode { + private static final long serialVersionUID = 1L; + + public MergeSampleActivityNode(ActivityId Id) { + super(Id); + } + + @Override + public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, + IRecordDescriptorProvider recordDescProvider, final int partition, final int nPartitions) + throws HyracksDataException { + return new MergeSampleOperatorNodePushable(ctx, new TaskId(getActivityId(), partition), sampleFields, + sampleBasis, frameLimit, recordDescProvider, outputLimit, inDesc, outDesc, + firstKeyNormalizerFactory, comparatorFactories, algorithm, partition, nPartitions); + } + } + + private class SampleReaderActivityNode extends AbstractActivityNode { + private static final long serialVersionUID = 1L; + + public SampleReaderActivityNode(ActivityId Id) { + super(Id); + } + + @Override + public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx, + IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions) + throws HyracksDataException { + return new AbstractUnaryOutputSourceOperatorNodePushable() { + @Override + public void initialize() throws HyracksDataException { + + } + + @Override + public void deinitialize() throws HyracksDataException { + + } + }; + } + } +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/MergeSampleOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/MergeSampleOperatorNodePushable.java new file mode 100644 index 00000000000..5f55ddd017a --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/MergeSampleOperatorNodePushable.java @@ -0,0 +1,151 @@ +/* + * 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.hyracks.dataflow.std.parallel.histogram; + +import java.nio.ByteBuffer; + +import org.apache.hyracks.api.context.IHyracksTaskContext; +import org.apache.hyracks.api.dataflow.value.IBinaryComparator; +import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory; +import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory; +import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider; +import org.apache.hyracks.api.dataflow.value.RecordDescriptor; +import org.apache.hyracks.api.exceptions.HyracksDataException; +import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable; +import org.apache.hyracks.dataflow.std.parallel.HistogramAlgorithm; +import org.apache.hyracks.dataflow.std.parallel.base.AbstractSamplingWriter; +import org.apache.hyracks.dataflow.std.parallel.base.MergeOrderedSampleWriter; +import org.apache.hyracks.dataflow.std.sort.Algorithm; +import org.apache.hyracks.dataflow.std.sort.FrameSorterMergeSort; +import org.apache.hyracks.dataflow.std.sort.FrameSorterQuickSort; +import org.apache.hyracks.dataflow.std.sort.IFrameSorter; +import org.apache.hyracks.dataflow.std.buffermanager.EnumFreeSlotPolicy; +import org.apache.hyracks.dataflow.std.buffermanager.FrameFreeSlotPolicyFactory; +import org.apache.hyracks.dataflow.std.buffermanager.IFrameBufferManager; +import org.apache.hyracks.dataflow.std.buffermanager.IFrameFreeSlotPolicy; +import org.apache.hyracks.dataflow.std.buffermanager.VariableFrameMemoryManager; +import org.apache.hyracks.dataflow.std.buffermanager.VariableFramePool; + +/** + * @author michael + */ +public class MergeSampleOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable { + private final IHyracksTaskContext ctx; + private final EnumFreeSlotPolicy policy = EnumFreeSlotPolicy.BIGGEST_FIT; + private final HistogramAlgorithm sortAlg = HistogramAlgorithm.ORDERED_HISTOGRAM; + private final IFrameSorter frameSorter; + private final Object stateId; + private final int frameLimit; + private final int outputLimit; + private final int[] sampleFields; + private final int sampleBasis; + private final HistogramAlgorithm algorithm; + private final RecordDescriptor inDesc; + private final RecordDescriptor outDesc; + private final IBinaryComparatorFactory[] comparatorFactories; + private final INormalizedKeyComputerFactory firstKeyNormalizerFactory; + private AbstractSamplingWriter sw; + + /*private MaterializingSampleTaskState state;*/ + + /** + * @throws HyracksDataException + */ + public MergeSampleOperatorNodePushable(final IHyracksTaskContext ctx, Object stateId, int[] sampleFields, + int sampleBasis, int frameLimit, IRecordDescriptorProvider recordDescProvider, int outputLimit, + RecordDescriptor inDesc, RecordDescriptor outDesc, INormalizedKeyComputerFactory firstKeyNormalizerFactory, + IBinaryComparatorFactory[] comparatorFactories, HistogramAlgorithm alg, final int partition, + final int nPartitions) throws HyracksDataException { + this.ctx = ctx; + this.stateId = stateId; + this.frameLimit = frameLimit; + this.outputLimit = outputLimit; + this.inDesc = inDesc; + this.outDesc = outDesc; + this.comparatorFactories = comparatorFactories; + this.firstKeyNormalizerFactory = firstKeyNormalizerFactory; + this.sampleFields = sampleFields; + this.sampleBasis = sampleBasis; + this.algorithm = alg; + IFrameFreeSlotPolicy freeSlotPolicy = null; + switch (policy) { + case BIGGEST_FIT: + freeSlotPolicy = FrameFreeSlotPolicyFactory.createFreeSlotPolicy(EnumFreeSlotPolicy.BIGGEST_FIT, + frameLimit - 1); + break; + case SMALLEST_FIT: + freeSlotPolicy = FrameFreeSlotPolicyFactory.createFreeSlotPolicy(EnumFreeSlotPolicy.SMALLEST_FIT); + break; + case LAST_FIT: + freeSlotPolicy = FrameFreeSlotPolicyFactory.createFreeSlotPolicy(EnumFreeSlotPolicy.LAST_FIT, + frameLimit - 1); + break; + } + IFrameBufferManager bufferManager = new VariableFrameMemoryManager(new VariableFramePool(ctx, (frameLimit - 1) + * ctx.getInitialFrameSize()), freeSlotPolicy); + if (sortAlg == HistogramAlgorithm.ORDERED_HISTOGRAM) { + frameSorter = new FrameSorterMergeSort(ctx, bufferManager, sampleFields, firstKeyNormalizerFactory, + comparatorFactories, inDesc, outputLimit); + } else { + frameSorter = new FrameSorterQuickSort(ctx, bufferManager, sampleFields, firstKeyNormalizerFactory, + comparatorFactories, inDesc, outputLimit); + } + } + + @Override + public void open() throws HyracksDataException { + IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length]; + for (int i = 0; i < comparatorFactories.length; i++) { + comparators[i] = comparatorFactories[i].createBinaryComparator(); + } + /*writer.open();*/ + switch (algorithm) { + case ORDERED_HISTOGRAM: + sw = new MergeOrderedSampleWriter(ctx, sampleFields, sampleBasis, comparators, inDesc, outDesc, writer); + sw.open(); + /*state = new MaterializingSampleTaskState(ctx.getJobletContext().getJobId(), stateId); + state.open(ctx);*/ + break; + + case UNIFORM_HISTOGRAM: + case RANDOM_HISTOGRAM: + case WAVELET_HISTOGRAM: + break; + default: + break; + } + } + + @Override + public void nextFrame(ByteBuffer buffer) throws HyracksDataException { + sw.nextFrame(buffer); + } + + @Override + public void fail() throws HyracksDataException { + sw.fail(); + } + + @Override + public void close() throws HyracksDataException { + /*state.close();*/ + sw.close(); + } +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/SampleReaderOperatorNodePushable.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/SampleReaderOperatorNodePushable.java new file mode 100644 index 00000000000..9eabfdc8505 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/SampleReaderOperatorNodePushable.java @@ -0,0 +1,107 @@ +/* + * 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.hyracks.dataflow.std.parallel.histogram; + +import java.nio.ByteBuffer; + +import org.apache.hyracks.api.context.IHyracksTaskContext; +import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory; +import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory; +import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider; +import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer; +import org.apache.hyracks.api.dataflow.value.RecordDescriptor; +import org.apache.hyracks.api.exceptions.HyracksDataException; +import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable; + +/** + * @author michael + */ +public class SampleReaderOperatorNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable { + + private final IHyracksTaskContext ctx; + private final int[] sampleFields; + private final int sampleBasis; + private final int frameLimit; + private final int outputLimit; + private final INormalizedKeyComputerFactory firstKeyNormalizerFactory; + private final IBinaryComparatorFactory[] comparatorFactories; + private RecordDescriptor outDesc; + private RecordDescriptor inDesc; + + /** + * + */ + public SampleReaderOperatorNodePushable(final IHyracksTaskContext ctx, int[] sampleFields, int sampleBasis, + int frameLimit, IRecordDescriptorProvider recordDescProvider, int outputLimit, RecordDescriptor inDesc, + INormalizedKeyComputerFactory firstKeyNormalizerFactory, IBinaryComparatorFactory[] comparatorFactories, + final int partition, final int nPartitions) { + this.ctx = ctx; + this.sampleFields = sampleFields; + this.sampleBasis = sampleBasis; + this.frameLimit = frameLimit; + this.outputLimit = outputLimit; + this.firstKeyNormalizerFactory = firstKeyNormalizerFactory; + this.comparatorFactories = comparatorFactories; + this.inDesc = inDesc; + @SuppressWarnings("rawtypes") + ISerializerDeserializer[] outSchema = new ISerializerDeserializer[sampleFields.length]; + for (int i = 0; i < sampleFields.length; i++) { + outSchema[i] = inDesc.getFields()[sampleFields[i]]; + } + this.outDesc = new RecordDescriptor(outSchema); + } + + /* (non-Javadoc) + * @see org.apache.hyracks.api.comm.IFrameWriter#open() + */ + @Override + public void open() throws HyracksDataException { + // TODO Auto-generated method stub + + } + + /* (non-Javadoc) + * @see org.apache.hyracks.api.comm.IFrameWriter#nextFrame(java.nio.ByteBuffer) + */ + @Override + public void nextFrame(ByteBuffer buffer) throws HyracksDataException { + // TODO Auto-generated method stub + + } + + /* (non-Javadoc) + * @see org.apache.hyracks.api.comm.IFrameWriter#fail() + */ + @Override + public void fail() throws HyracksDataException { + // TODO Auto-generated method stub + + } + + /* (non-Javadoc) + * @see org.apache.hyracks.api.comm.IFrameWriter#close() + */ + @Override + public void close() throws HyracksDataException { + // TODO Auto-generated method stub + + } + +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/structures/AbstractStreamingHistogram.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/structures/AbstractStreamingHistogram.java new file mode 100644 index 00000000000..8ea7d0f6786 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/structures/AbstractStreamingHistogram.java @@ -0,0 +1,226 @@ +/* + * 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.hyracks.dataflow.std.parallel.histogram.structures; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map.Entry; + +import org.apache.hyracks.api.dataflow.value.IBinaryComparator; +import org.apache.hyracks.api.exceptions.HyracksDataException; +import org.apache.hyracks.data.std.api.AbstractPointable; +import org.apache.hyracks.data.std.api.IPointable; +import org.apache.hyracks.dataflow.std.parallel.IDTHistogram; +import org.apache.hyracks.dataflow.std.parallel.IHistogram; + +/** + * @author michael + */ +public class AbstractStreamingHistogram implements IDTHistogram { + // GrowableArray quantiles; + private static final int DEFAULT_WINDOW_FACTOR = 10; + private final IBinaryComparator[] comparators; + private final List> quantiles; + private final List buffered; + protected final double mu; + private final int elastic; + private final int buckets; + private final int blocked; + private final int windows; + protected final int granulars; + protected final int threshold; + + private int current = 0; + private int bucket = 0; + + public class Quantile implements Entry { + private K key; + private V value; + + public Quantile(K key, V value) { + this.key = key; + this.value = value; + } + + @Override + public K getKey() { + return key; + } + + @Override + public V getValue() { + return value; + } + + @Override + public V setValue(V value) { + V old = this.value; + this.value = value; + return old; + } + } + + public AbstractStreamingHistogram(IBinaryComparator[] comparators, int nu, int el, double mu) { + this.comparators = comparators; + this.mu = mu; + this.buckets = nu; + this.elastic = el; + this.blocked = elastic * buckets; + this.windows = blocked * DEFAULT_WINDOW_FACTOR; + this.threshold = (int) Math.floor((windows / blocked) * mu); + this.granulars = windows / blocked; + this.quantiles = new ArrayList>(blocked); + this.buffered = new ArrayList(); + } + + private int compareKey(E left, E right) throws HyracksDataException { + byte[] leftBuf = ((IPointable) left).getByteArray(); + byte[] rightBuf = ((IPointable) right).getByteArray(); + int ret = 0; + for (int i = 0; i < comparators.length; i++) { + ret = comparators[i].compare(leftBuf, 0, leftBuf.length, rightBuf, 0, rightBuf.length); + if (ret != 0) + break; + } + return 0; + } + + private void sortBuffered() { + Collections.sort(buffered, new Comparator() { + @Override + public int compare(E q1, E q2) { + try { + return compareKey(q1, q2); + } catch (HyracksDataException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + return Integer.MIN_VALUE; + } + }); + } + + public void sortQuantiles() { + Collections.sort(quantiles, new Comparator>() { + @Override + public int compare(Entry q1, Entry q2) { + try { + return compareKey(q1.getKey(), q2.getKey()); + } catch (HyracksDataException e) { + // TODO Auto-generated catch block + e.printStackTrace(); + } + return Integer.MIN_VALUE; + } + }); + } + + public int binarySearch(E next) throws HyracksDataException { + int low = 0; + int high = bucket; + while (high - low != 1) { + int middle = (high - low) / 2 + low; + if (compareKey(quantiles.get(middle).getKey(), next) > 0) + high = middle; + else + low = middle; + } + return low; + } + + public void updateIncreaseQuantile(int at, E quantile) { + quantiles.set(at, new Quantile(quantile, quantiles.get(at).getValue() + 1)); + } + + @Override + public E mediate(E left, E right) { + /*byte[] bm = new byte[left.length]; + double dm = (DoublePointable.getDouble(left, 0) + DoublePointable.getDouble(right, 0)) / 2; + DoublePointable.setDouble(bm, 0, dm); + return bm;*/ + return null; + } + + /*private int comparator(IHistogram left, IHistogram right) { + return 0; + } + + private void merge() { + + } + + private void split() { + + }*/ + + @Override + public int getCurrent() { + return current; + } + + @Override + public void merge(IHistogram ba) throws HyracksDataException { + /*int total = current + ba.getCurrent(); + current = 0;*/ + } + + @Override + public void addItem(E item) { + int diff = ++current - windows; + if (diff > 0) { + //Need to insert into quantiles; + } else if (diff == 0) { + //Need to initial quantiles in batch + sortBuffered(); + } else { + //Just insert into buffer + this.buffered.add(item); + } + } + + @Override + public void countItem(E item) { + + } + + @Override + public List> generate(boolean isGlobal) { + return quantiles; + } + + @Override + public void initialize() { + this.quantiles.clear(); + this.buffered.clear(); + } + + @Override + public void countReset() { + + } + + @Override + public FieldType getType() { + // TODO Auto-generated method stub + return null; + } +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/structures/DTStreamingHistogram.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/structures/DTStreamingHistogram.java new file mode 100644 index 00000000000..9db5c4d0f1e --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/structures/DTStreamingHistogram.java @@ -0,0 +1,1018 @@ +/* + * 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.hyracks.dataflow.std.parallel.histogram.structures; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map.Entry; +import java.util.PriorityQueue; +import java.util.logging.Logger; +import java.util.Random; + +import org.apache.hyracks.api.exceptions.HyracksDataException; +import org.apache.hyracks.data.std.api.AbstractPointable; +import org.apache.hyracks.dataflow.std.parallel.IDTHistogram; +import org.apache.hyracks.dataflow.std.parallel.IHistogram; +import org.apache.hyracks.dataflow.std.parallel.util.HistogramUtils; + +/** + * @author michael + */ +public class DTStreamingHistogram implements IDTHistogram { + private static final Logger LOGGER = Logger.getLogger(DTStreamingHistogram.class.getName()); + + private boolean heapIncrement = false; + + private/*static final*/boolean equalHeight = false; + + private/*static final*/int QUANTILE_SCALE = 8; + + private final boolean generateWithoutReduce = false; + + private boolean adjustedBound = false; + + private final BOUNDARY_TYPE boundary = BOUNDARY_TYPE.ACCUMULATED; + + //private static final boolean streamMerge = false; + + private static final int DEFAULT_ANSI_SAMPLE_LENGTH = 9; + + private double leftmostItem; + + private double rightmostItem; + + private final boolean minMaxReproduction = true; + + private enum BOUNDARY_TYPE { + MEDIATE, + INTERPOLATE, + ACCUMULATED, + RAW + } + + private class Coord implements Comparable { + double x; + int y; + + @SuppressWarnings("unchecked") + public int compareTo(Object other) { + return Double.compare(x, ((Coord) other).x); + } + }; + + private double pointableToQuantile(E item) throws HyracksDataException { + switch (type) { + case SHORT: + return HistogramUtils.shortMappingToQuantile(item); + case INT: + return HistogramUtils.integerMappingToQuantile(item); + case LONG: + return HistogramUtils.longMappingToQuantile(item); + case FLOAT: + return HistogramUtils.floatMappingToQuantile(item); + case DOUBLE: + return HistogramUtils.doubleMappingToQuantile(item); + case UTF8: + return HistogramUtils.ansiMappingToQuantile(item, 0, DEFAULT_ANSI_SAMPLE_LENGTH); + default: + throw new HyracksDataException("Type " + item.getClass() + " cannot be supported."); + } + } + + @SuppressWarnings("unchecked") + private E quantileToPointable(double d) throws HyracksDataException { + switch (type) { + case SHORT: + return (E) HistogramUtils.quantileRevertToShort(d); + case INT: + return (E) HistogramUtils.quantileRevertToInteger(d); + case LONG: + return (E) HistogramUtils.quantileRevertToLong(d); + case FLOAT: + return (E) HistogramUtils.quantileRevertToFloat(d); + case DOUBLE: + return (E) HistogramUtils.quantileRevertToDouble(d); + case UTF8: + return (E) HistogramUtils.quantileRevertToAnsi(d, DEFAULT_ANSI_SAMPLE_LENGTH); + default: + throw new HyracksDataException("Type enum " + type + " cannot be supported."); + } + } + + private int current; + private int nbins; + private int nusedbins; + private List bins; + private Random prng; + private FieldType type; + /*private BoundedPriorityQueue domQuantiles = null;*/ + public DominantQuantile peakTest = new DominantQuantile(0, 0); + private DominantQuantile heapHead; + public long updateHeap = 0; + + public class DominantQuantile implements Comparable> { + T dominant; + int iBin; + + public DominantQuantile(T dom, int iBin) { + this.dominant = dom; + this.iBin = iBin; + } + + public void setBin(int bin) { + this.iBin = bin; + } + + public void setDome(T dom) { + this.dominant = dom; + } + + public void update(T dom, int bin) { + this.iBin = bin; + this.dominant = dom; + } + + public int getBin() { + return iBin; + } + + public T getDom() { + return dominant; + } + + @Override + public int compareTo(DominantQuantile o) { + // TODO Auto-generated method stub + return Double.compare(dominant.doubleValue(), o.getDom().doubleValue()); + } + } + + public class BoundedPriorityQueue extends PriorityQueue> { + private static final long serialVersionUID = 1L; + private int limit; + private DominantQuantile peek; + private boolean asc = true; + + public BoundedPriorityQueue(int maxCapacity, boolean asc) { + this.limit = maxCapacity; + this.asc = asc; + } + + //@SuppressWarnings({ "unchecked", "rawtypes" }) + @Override + public boolean add(DominantQuantile e) { + if (super.size() < limit) { + boolean ret = super.add(e); + peek = super.peek(); + return ret; + } else { + if (asc) { + if (peek.compareTo(e) < 0) { + super.remove(); + boolean ret = super.add(e); + peek = super.peek(); + return ret; + } + } else { + if (peek.compareTo(e) > 0) { + super.remove(); + boolean ret = super.add(e); + peek = super.peek(); + return ret; + } + } + } + return false; + } + + public int getLimit() { + return limit; + } + } + + public DTStreamingHistogram() { + nbins = 0; + nusedbins = 0; + bins = null; + prng = new Random(31183); + } + + public DTStreamingHistogram(FieldType t) { + this(); + type = t; + } + + public DTStreamingHistogram(FieldType t, boolean heapActive) { + this(); + this.type = t; + this.heapIncrement = heapActive; + } + + public boolean isReady() { + return (getCurrent() != 0); + } + + public int getNBins() { + return nbins; + } + + public int getNUsedBins() { + return nusedbins; + } + + public List getBins() { + return bins; + } + + public Coord getBin(int b) { + return bins.get(b); + } + + public void allocate(int num_bins) { + nbins = num_bins * QUANTILE_SCALE; + if (heapIncrement) + bins = new ArrayList(); + else + bins = new ArrayList(); + nusedbins = 0; + leftmostItem = Double.MAX_VALUE; + rightmostItem = Double.MIN_VALUE; + } + + public void allocate(int num_bins, int scale, boolean equalHeight) { + this.QUANTILE_SCALE = scale; + this.equalHeight = equalHeight; + if (heapIncrement) { + if (equalHeight) { + //this.domQuantiles = new BoundedPriorityQueue(1, true); + heapHead = new DominantQuantile(Integer.MAX_VALUE, -1); + } else { + //this.domQuantiles = new BoundedPriorityQueue(1, false); + heapHead = new DominantQuantile(Double.MAX_VALUE, -1); + } + } + allocate(num_bins); + } + + public FieldType getType() { + return type; + } + + public E quantile(double q) throws HyracksDataException { + assert (bins != null && nusedbins > 0 && nbins > 0); + double sum = 0, csum = 0; + int b; + for (b = 0; b < nusedbins; b++) { + sum += bins.get(b).y; + } + for (b = 0; b < nusedbins; b++) { + csum += bins.get(b).y; + if (csum / sum >= q) { + if (b == 0) { + E ret = quantileToPointable(bins.get(b).x); + return ret; + } + + csum -= bins.get(b).y; + double r = bins.get(b - 1).x + (q * sum - csum) * (bins.get(b).x - bins.get(b - 1).x) / (bins.get(b).y); + + E ret = quantileToPointable(r); + return ret; + } + } + return null; + } + + public void trim() { + if (equalHeight) + trimForHeight(); + else + trimForWidth(); + } + + public void trimForWidth() { + while (nusedbins > nbins) { + double smallestdiff = bins.get(1).x - bins.get(0).x; + int smallestdiffloc = 0; + int smallestdiffcount = 1; + for (int i = 1; i < nusedbins - 1; i++) { + double diff = bins.get(i + 1).x - bins.get(i).x; + if (diff < smallestdiff) { + smallestdiff = diff; + smallestdiffloc = i; + smallestdiffcount = 1; + } else { + if (diff == smallestdiff && prng.nextDouble() <= (1.0 / ++smallestdiffcount)) { + smallestdiffloc = i; + } + } + } + int d = bins.get(smallestdiffloc).y + bins.get(smallestdiffloc + 1).y; + Coord smallestdiffbin = bins.get(smallestdiffloc); + smallestdiffbin.x *= (double) smallestdiffbin.y / d; + smallestdiffbin.x += bins.get(smallestdiffloc + 1).x / d * bins.get(smallestdiffloc + 1).y; + smallestdiffbin.y = d; + peakTest.update(smallestdiff, smallestdiffloc); + bins.remove(smallestdiffloc + 1); + nusedbins--; + updateHeap++; + } + } + + public void trimForHeight() { + while (nusedbins > nbins) { + int maxHeightSum = bins.get(1).y + bins.get(0).y; + int maxHeightLoc = 0; + int smallestdiffcount = 1; + for (int i = 1; i < nusedbins - 1; i++) { + int curHeightSum = bins.get(i + 1).y + bins.get(i).y; + if (curHeightSum < maxHeightSum) { + maxHeightSum = curHeightSum; + maxHeightLoc = i; + smallestdiffcount = 1; + } else { + if (curHeightSum == maxHeightSum && prng.nextDouble() <= (1.0 / ++smallestdiffcount)) { + maxHeightLoc = i; + } + } + } + int d = bins.get(maxHeightLoc).y + bins.get(maxHeightLoc + 1).y; + Coord smallestdiffbin = bins.get(maxHeightLoc); + smallestdiffbin.x *= (double) smallestdiffbin.y / d; + smallestdiffbin.x += bins.get(maxHeightLoc + 1).x / d * bins.get(maxHeightLoc + 1).y; + smallestdiffbin.y = d; + peakTest.update(maxHeightSum, maxHeightLoc); + bins.remove(maxHeightLoc + 1); + nusedbins--; + updateHeap++; + } + } + + /* (non-Javadoc) + * @see org.apache.hyracks.dataflow.std.sample.IHistogram#initialize() + */ + @Override + public void initialize() { + // TODO Auto-generated method stub + bins = null; + nbins = nusedbins = 0; + } + + /* (non-Javadoc) + * @see org.apache.hyracks.dataflow.std.sample.IHistogram#merge(org.apache.hyracks.dataflow.std.sample.IHistogram) + */ + @Override + public void merge(IHistogram ba) throws HyracksDataException { + if (null == ba) { + return; + } + + if (!(ba instanceof DTStreamingHistogram)) { + LOGGER.info("Sampling error: " + ba.getCurrent()); + throw new HyracksDataException("Failed to get the proper sampling bins."); + } else if (type != ba.getType()) + throw new HyracksDataException("Mismatching hitogram type."); + + DTStreamingHistogram other = (DTStreamingHistogram) ba; + if (nbins == 0 || nusedbins == 0) { + nbins = other.getNBins(); + nusedbins = other.getNUsedBins(); + bins = new ArrayList(nusedbins); + for (int i = 0; i < nusedbins; i++) { + Coord bin = new Coord(); + bin.x = other.getBin(i).x; + bin.y = other.getBin(i).y; + bins.add(bin); + } + } else { + List tmpbins = new ArrayList(nusedbins + other.getNUsedBins()); + for (int i = 0; i < nusedbins; i++) { + Coord bin = new Coord(); + bin.x = bins.get(i).x; + bin.y = bins.get(i).y; + tmpbins.add(bin); + } + for (int i = 0; i < other.getBins().size(); i++) { + Coord bin = new Coord(); + bin.x = other.getBin(i).x; + bin.y = other.getBin(i).y; + bins.add(bin); + } + Collections.sort(tmpbins); + bins = tmpbins; + nusedbins += other.getNBins(); + trim(); + } + } + + private double quantileMerge(double q1, double q2, int k1, int k2) { + double q = .0; + q = (q1 * k1 + q2 * k2) / (k1 + k2); + return q; + } + + private void selectedBinUpdateByOne(int requirePoint, double q) throws HyracksDataException { + int d = bins.get(requirePoint).y + 1; + Coord selectedMergeBin = bins.get(requirePoint); + selectedMergeBin.x *= (double) selectedMergeBin.y / d; + selectedMergeBin.x += q / d; + selectedMergeBin.y = d; + } + + private void selectedBinMergeWithNext(int mergingPoint) throws HyracksDataException { + Coord mergeLeft = bins.get(mergingPoint); + int d = bins.get(mergingPoint).y + bins.get(mergingPoint + 1).y; + mergeLeft.x *= (double) mergeLeft.y / d; + mergeLeft.x += bins.get(mergingPoint + 1).x / d * bins.get(mergingPoint + 1).y; + mergeLeft.y = d; + bins.remove(mergingPoint + 1); + } + + private void atomicInsert(int bin, double q) throws HyracksDataException { + if (equalHeight) { + int requirePoint = -1; + if (bin > 0 && bin <= nusedbins - 1) { + requirePoint = (bins.get(bin - 1).y > bins.get(bin).y) ? bin : (bin - 1); + } else if (bin == 0) { + requirePoint = 0; + } else if (bin == nusedbins) { + requirePoint = nusedbins - 1; + } else { + throw new HyracksDataException("Invalid required position for minSum: " + bin + " out of " + nusedbins); + } + if (requirePoint == heapHead.getBin() || requirePoint - 1 == heapHead.getBin()) { + //before: [a1], [1], [a2]; + //after: case1:[a1 + 1], [a2]; case2: [a1], [a2 + 1]; both violate the peak limit of heap; + selectedBinUpdateByOne(requirePoint, q); + sequentialScanAndUpdatePeak(); + } else if (heapHead.getDom().intValue() >= bins.get(requirePoint).y + 1) { + //the most common case: merge 1 with requirePoint and keep heap unchanged. + selectedBinUpdateByOne(requirePoint, q); + } else { + //merge the heap point and insert the [q, 1] into the bins. + Coord newBin = new Coord(); + newBin.x = q; + newBin.y = 1; + bins.add(bin, newBin); + if (heapHead.getBin() >= bin) + heapHead.setBin(heapHead.getBin() + 1); + selectedBinMergeWithNext(heapHead.getBin()); + sequentialScanAndUpdatePeak(); + } + } else { + int requirePoint = -1; + double expectedMinDiff = .0; + boolean expectedLeftMerge = true; + if (bin > 0 && bin <= nusedbins - 1) { + if (q - bins.get(bin - 1).x > bins.get(bin).x - q) { + expectedLeftMerge = false; + expectedMinDiff = quantileMerge(q, bins.get(bin).x, 1, bins.get(bin).y) - bins.get(bin - 1).x; + requirePoint = bin; + } else { + expectedLeftMerge = true; + expectedMinDiff = bins.get(bin).x - quantileMerge(bins.get(bin - 1).x, q, bins.get(bin - 1).y, 1); + requirePoint = bin - 1; + } + } else if (bin == 0) { + expectedLeftMerge = false; + expectedMinDiff = bins.get(1).x - quantileMerge(q, bins.get(0).x, 1, bins.get(0).y); + requirePoint = 0; + } else if (bin == nusedbins) { + expectedLeftMerge = true; + expectedMinDiff = quantileMerge(bins.get(nusedbins - 1).x, q, bins.get(nusedbins - 1).y, 1) + - bins.get(nusedbins - 1).x; + requirePoint = nusedbins - 1; + } else + throw new HyracksDataException("Invalid required position for minDiff: " + bin + " out of " + nusedbins); + if (!expectedLeftMerge && requirePoint == heapHead.getBin() || expectedLeftMerge + && requirePoint - 1 == heapHead.getBin()) { + selectedBinUpdateByOne(requirePoint, q); + sequentialScanAndUpdatePeak(); + } else if (heapHead.getDom().doubleValue() >= expectedMinDiff) { + selectedBinUpdateByOne(requirePoint, q); + if (requirePoint != 0 && requirePoint != nusedbins - 1) + heapHead.update(expectedMinDiff, requirePoint); + } else { + Coord newBin = new Coord(); + newBin.x = q; + newBin.y = 1; + bins.add(bin, newBin); + if (heapHead.getBin() >= bin) + heapHead.setBin(heapHead.getBin() + 1); + selectedBinMergeWithNext(heapHead.getBin()); + sequentialScanAndUpdatePeak(); + } + } + } + + private void sequentialScanAndUpdatePeak() throws HyracksDataException { + updateHeap++; + if (equalHeight) { + int minHeightSum = bins.get(1).y + bins.get(0).y; + int minHeightLoc = 0; + int smallestdiffcount = 1; + for (int i = 1; i < nusedbins - 1; i++) { + int curHeightSum = bins.get(i + 1).y + bins.get(i).y; + if (curHeightSum < minHeightSum) { + minHeightSum = curHeightSum; + minHeightLoc = i; + smallestdiffcount = 1; + } else { + if (curHeightSum == minHeightSum && prng.nextDouble() <= (1.0 / ++smallestdiffcount)) { + minHeightLoc = i; + } + } + } + heapHead.update(minHeightSum, minHeightLoc); + } else { + double minDiffQut = bins.get(1).x - bins.get(0).x; + int minDiffLoc = 0; + int smallestdiffcount = 1; + for (int i = 1; i < nusedbins - 1; i++) { + double curDiffQut = bins.get(i + 1).x - bins.get(i).x; + if (curDiffQut < minDiffQut) { + minDiffQut = curDiffQut; + minDiffLoc = i; + smallestdiffcount = 1; + } else { + if (curDiffQut == minDiffQut && prng.nextDouble() <= (1.0 / ++smallestdiffcount)) { + minDiffLoc = i; + } + } + } + heapHead.update(minDiffQut, minDiffLoc); + } + } + + private void updateHeap(int requirePoint) { + if (equalHeight) { + if (requirePoint > 0 && requirePoint < nusedbins - 1) { + Integer dom = bins.get(requirePoint - 1).y + bins.get(requirePoint).y; + if (heapHead.getDom().intValue() > dom) + heapHead.update(dom, requirePoint - 1); + dom = bins.get(requirePoint).y + bins.get(requirePoint + 1).y; + if (heapHead.getDom().intValue() > dom) + heapHead.update(dom, requirePoint); + } else if (requirePoint == 0) { + Integer dom = bins.get(0).y + bins.get(1).y; + if (heapHead.getDom().intValue() > dom) + heapHead.update(dom, 0); + } else if (requirePoint == nusedbins - 1) { + Integer dom = bins.get(nusedbins - 2).y + bins.get(nusedbins - 1).y; + if (heapHead.getDom().intValue() > dom) + heapHead.update(dom, nusedbins - 2); + } + } else { + if (requirePoint > 0 && requirePoint < nusedbins - 1) { + Double dom = bins.get(requirePoint).y - bins.get(requirePoint - 1).x; + if (heapHead.getDom().doubleValue() > dom) + heapHead.update(dom, requirePoint - 1); + dom = bins.get(requirePoint + 1).x - bins.get(requirePoint).x; + if (heapHead.getDom().doubleValue() > dom) + heapHead.update(dom, requirePoint); + } else if (requirePoint == 0) { + Double dom = bins.get(1).x - bins.get(0).x; + if (heapHead.getDom().doubleValue() > dom) + heapHead.update(dom, 0); + } else if (requirePoint == nusedbins - 1) { + Double dom = bins.get(nusedbins - 1).x - bins.get(nusedbins - 2).x; + if (heapHead.getDom().doubleValue() > dom) + heapHead.update(dom, nusedbins - 2); + } + } + } + + /* (non-Javadoc) + * @see org.apache.hyracks.dataflow.std.sample.IHistogram#addItem(byte[]) + */ + @Override + public void addItem(E item) throws HyracksDataException { + double q = pointableToQuantile(item); + if (q < leftmostItem) + leftmostItem = q; + if (q > rightmostItem) + rightmostItem = q; + int bin = 0; + for (int l = 0, r = nusedbins; l < r;) { + bin = (l + r) / 2; + if (bins.get(bin).x > q) { + r = bin; + } else { + if (bins.get(bin).x < q) { + l = ++bin; + } else { + break; + } + } + } + if (bin < nusedbins && bins.get(bin).x == q) { + bins.get(bin).y++; + } else { + if (heapIncrement) { + if (nusedbins < nbins) { + Coord newBin = new Coord(); + newBin.x = q; + newBin.y = 1; + bins.add(bin, newBin); + nusedbins++; + if (nusedbins > 1) { + if (heapHead.getBin() >= bin) + heapHead.setBin(heapHead.getBin() + 1); + updateHeap(bin); + } + } else { + atomicInsert(bin, q); + } + } else { + Coord newBin = new Coord(); + newBin.x = q; + newBin.y = 1; + bins.add(bin, newBin); + if (++nusedbins > nbins) { + trim(); + } + } + } + } + + public class Quantile implements Entry { + private K key; + private V value; + + public Quantile(K key, V value) { + this.key = key; + this.value = value; + } + + @Override + public K getKey() { + return key; + } + + @Override + public V getValue() { + return value; + } + + @Override + public V setValue(V value) { + V old = this.value; + this.value = value; + return old; + } + } + + /* (non-Javadoc) + * @see org.apache.hyracks.dataflow.std.sample.IHistogram#getCurrent() + */ + @Override + public int getCurrent() { + // TODO Auto-generated method stub + return current; + } + + private double[] leftExtend(double leftX, double leftY, double rightX, double rightY) { + double pointX = 0; + double pointY = 0; + if (minMaxReproduction) { + pointY = 1; + pointX = leftmostItem; + } else { + pointY = 2 * leftY - rightY; + pointX = 2 * leftX - rightX; + } + double point[] = new double[2]; + point[0] = pointX; + if (minMaxReproduction) + point[1] = pointY; + else + point[1] = 0; + return point; + } + + private double[] rightExtend(double leftX, double leftY, double rightX, double rightY) { + double pointY = 0; + double pointX = 0; + if (minMaxReproduction) { + pointY = 1; + pointX = rightmostItem; + } else { + pointY = 2 * rightY - leftY; + pointX = 2 * rightX - leftX; + } + double point[] = new double[2]; + point[0] = pointX; + if (minMaxReproduction) + point[1] = pointY; + else + point[1] = 0; + return point; + } + + private double[] accumulate(double want, double leftX, double leftY, double rightX, double rightY, double localX, + double elipsed) { + double localY = leftY + (rightY - leftY) * (localX - leftX) / (rightX - leftX); + double pointY = Math.sqrt(localY * localY + 2 * want * (rightY - leftY)); + double pointX = localX + 2 * (rightX - leftX) / (pointY + localY) * want; + double point[] = new double[2]; + point[0] = pointX; + point[1] = /*pointY*/want; + return point; + } + + /* (non-Javadoc) + * @see org.apache.hyracks.dataflow.std.sample.IHistogram#generate() + */ + @Override + public List> generate(boolean isGlobal) throws HyracksDataException { + List> ret = new ArrayList>(); + if (adjustedBound) { + for (int i = 0; i < nusedbins; i++) { + E pQuan = quantileToPointable(bins.get(i).x); + ret.add(new Quantile(pQuan, bins.get(i).y)); + } + } else { + switch (boundary) { + case MEDIATE: { + for (int i = 0; i < nusedbins - 1; i++) { + E pQuan = mediate(quantileToPointable(bins.get(i).x), quantileToPointable(bins.get(i + 1).x)); + ret.add(new Quantile(pQuan, bins.get(i).y)); + } + ret.add(new Quantile(quantileToPointable(Double.MAX_VALUE), bins.get(nbins - 1).y)); + break; + } + case INTERPOLATE: { + if (generateWithoutReduce) { + for (int i = 0; i < nusedbins - 1; i++) { + E pQuan = mediate(quantileToPointable(bins.get(i).x), + quantileToPointable(bins.get(i + 1).x)); + ret.add(new Quantile(pQuan, bins.get(i).y)); + } + ret.add(new Quantile(quantileToPointable(Double.MAX_VALUE), bins.get(nbins - 1).y)); + break; + } else { + int count = 0; + for (int i = 0; i < nusedbins - 1; i++) { + count += bins.get(i).y; + if ((i + 1) % QUANTILE_SCALE == 0) { + Coord cod = interpolate(bins.get(i), bins.get(i + 1)); + ret.add(new Quantile(quantileToPointable(cod.x), count)); + count = 0; + } + } + ret.add(new Quantile(quantileToPointable(Double.MAX_VALUE), count + + bins.get(nbins - 1).y)); + break; + } + } + case ACCUMULATED: { + List cacheBins = new ArrayList(); + cacheBins.addAll(bins); + long total = 0; + for (int i = 0; i < nusedbins; i++) + total += cacheBins.get(i).y; + double[] leftVirtual = leftExtend(cacheBins.get(0).x, cacheBins.get(0).y, cacheBins.get(1).x, + cacheBins.get(1).y); + double[] rightVirtual = rightExtend(cacheBins.get(nusedbins - 2).x, cacheBins.get(nusedbins - 2).y, + cacheBins.get(nusedbins - 1).x, cacheBins.get(nusedbins - 1).y); + Coord leftExt = new Coord(); + leftExt.x = leftVirtual[0]; + leftExt.y = (int) leftVirtual[1]; + cacheBins.add(0, leftExt); + Coord rightExt = new Coord(); + rightExt.x = rightVirtual[0]; + rightExt.y = (int) rightVirtual[1]; + cacheBins.add(rightExt); + + int nParts = nusedbins / QUANTILE_SCALE; + double expection = (double) total / nParts; + double accd = .0; + int current = 0; + double localX = bins.get(0).x; + double elipsed = .0; + for (int i = 0; i < nParts - 1; i++) { + Coord cur = new Coord(); + while (true) { + if (current == cacheBins.size() - 1) + break; + if ((double) (cacheBins.get(current).y + cacheBins.get(current + 1).y) / 2 - elipsed > expection + - accd) { + double[] quan = accumulate(expection - accd, cacheBins.get(current).x, + cacheBins.get(current).y, cacheBins.get(current + 1).x, + cacheBins.get(current + 1).y, localX, elipsed); + cur.x = quan[0]; + cur.y = (int) expection; + localX = quan[0]; + elipsed += quan[1]; + ret.add(new Quantile(quantileToPointable(cur.x), cur.y)); + accd = 0; + break; + } else if ((double) (cacheBins.get(current).y + cacheBins.get(current + 1).y) / 2 - elipsed == expection + - accd) { + ret.add(new Quantile(quantileToPointable(cacheBins.get(current + 1).x), + (int) expection)); + current++; + localX = bins.get(current).x; + elipsed = .0; + accd = 0; + break; + } else { + accd += (double) (cacheBins.get(current).y + cacheBins.get(current + 1).y) / 2 + - elipsed; + current++; + localX = bins.get(current).x; + elipsed = .0; + } + } + } + ret.add(new Quantile(quantileToPointable(Double.MAX_VALUE), (int) expection)); + break; + } + case RAW: { + for (int i = 0; i < nusedbins; i++) { + E pQuan = quantileToPointable(bins.get(i).x); + ret.add(new Quantile(pQuan, bins.get(i).y)); + } + break; + } + } + } + return ret; + } + + @Override + public void countItem(E item) throws HyracksDataException { + double q = pointableToQuantile(item); + int bin = 0; + for (int l = 0, r = nusedbins; l < r;) { + bin = (l + r) / 2; + if (bins.get(bin).x > q) { + r = bin; + } else { + if (bins.get(bin).x < q) { + l = ++bin; + } else { + break; + } + } + } + int mark = 0; + if (bin == nusedbins) { + mark = bin - 1; + } else { + mark = bin; + } + bins.get(mark).y += 1; + } + + @Override + public void countReset() throws HyracksDataException { + if (!adjustedBound) { + switch (boundary) { + case MEDIATE: { + for (int i = 0; i < nusedbins - 1; i++) { + bins.get(i).x = (bins.get(i).x + bins.get(i + 1).x) / 2; + } + bins.get(bins.size() - 1).x = Double.MAX_VALUE; + break; + } + case INTERPOLATE: { + for (int i = 0; i < nusedbins - 1; i++) { + Coord ret = interpolate(bins.get(i), bins.get(i + 1)); + bins.get(i).x = ret.x; + bins.get(i).y = ret.y; + } + bins.get(bins.size() - 1).x = Double.MAX_VALUE; + Iterator iter = bins.iterator(); + int i = 0; + while (iter.hasNext()) { + iter.next(); + if ((i++ + 1) % QUANTILE_SCALE != 0) + iter.remove(); + } + nusedbins = bins.size(); + nbins = bins.size(); + break; + } + case ACCUMULATED: { + long total = 0; + for (int i = 0; i < nusedbins; i++) + total += bins.get(i).y; + double[] leftVirtual = leftExtend(bins.get(0).x, bins.get(0).y, bins.get(1).x, bins.get(1).y); + double[] rightVirtual = rightExtend(bins.get(nusedbins - 2).x, bins.get(nusedbins - 2).y, + bins.get(nusedbins - 1).x, bins.get(nusedbins - 1).y); + Coord leftExt = new Coord(); + leftExt.x = leftVirtual[0]; + leftExt.y = (int) leftVirtual[1]; + bins.add(0, leftExt); + Coord rightExt = new Coord(); + rightExt.x = rightVirtual[0]; + rightExt.y = (int) rightVirtual[1]; + bins.add(rightExt); + + /*for (int i = 0; i < bins.size(); i++) + LOGGER.info("<" + bins.get(i).x + ", " + bins.get(i).y + ">");*/ + + int nParts = nusedbins / QUANTILE_SCALE; + double expection = (double) total / nParts; + /*LOGGER.info("Total: " + total + " avg: " + expection + " parts: " + nParts);*/ + List gBins = new ArrayList(); + double accd = .0; + int current = 0; + double localX = bins.get(0).x; + double elipsed = .0; + for (int i = 0; i < nParts - 1; i++) { + Coord cur = new Coord(); + while (true) { + if (current == bins.size() - 1) + break; + if ((double) (bins.get(current).y + bins.get(current + 1).y) / 2 - elipsed > expection + - accd) { + double[] quan = accumulate(expection - accd, bins.get(current).x, bins.get(current).y, + bins.get(current + 1).x, bins.get(current + 1).y, localX, elipsed); + cur.x = quan[0]; + cur.y = (int) expection; + localX = quan[0]; + elipsed += quan[1]; + gBins.add(cur); + /*LOGGER.info("x: " + cur.x + " y: " + cur.y);*/ + accd = 0; + break; + } else if ((double) (bins.get(current).y + bins.get(current + 1).y) / 2 - elipsed == expection + - accd) { + gBins.add(bins.get(current + 1)); + gBins.get(gBins.size() - 1).y = (int) expection; + /*LOGGER.info("*x: " + gBins.get(gBins.size() - 1).x + " y: " + + gBins.get(gBins.size() - 1).y);*/ + current++; + localX = bins.get(current).x; + elipsed = .0; + accd = 0; + break; + } else { + accd += (double) (bins.get(current).y + bins.get(current + 1).y) / 2 - elipsed; + current++; + localX = bins.get(current).x; + elipsed = .0; + } + } + } + /*for (int i = 0; i < bins.size(); i++) + LOGGER.info("<" + bins.get(i).x + ", " + bins.get(i).y + ">");*/ + gBins.add(new Coord()); + gBins.get(gBins.size() - 1).x = Double.MAX_VALUE; + gBins.get(gBins.size() - 1).y = (int) expection; + bins.clear(); + bins.addAll(gBins); + nusedbins = bins.size(); + nbins = bins.size(); + break; + } + case RAW: + break; + } + adjustedBound = true; + } + for (int i = 0; i < nusedbins; i++) { + bins.get(i).y = 0; + } + } + + @Override + public E mediate(E left, E right) throws HyracksDataException { + return quantileToPointable((pointableToQuantile(left) + pointableToQuantile(right)) / 2); + } + + private Coord interpolate(Coord left, Coord right) { + //Currently, we support equal height histogram. + Coord ret = new Coord(); + if (equalHeight) { + ret.x = (left.x + right.x) / 2; + ret.y = left.y; + } else { + //To be continued. + ret.x = left.x; + ret.y = left.y; + } + return ret; + } +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/structures/TernaryIterativeHistogram.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/structures/TernaryIterativeHistogram.java new file mode 100644 index 00000000000..f6278ea1927 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/structures/TernaryIterativeHistogram.java @@ -0,0 +1,407 @@ +/* + * 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.hyracks.dataflow.std.parallel.histogram.structures; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.logging.Logger; + +import org.apache.hyracks.api.exceptions.HyracksDataException; +import org.apache.hyracks.data.std.primitive.UTF8StringPointable; +import org.apache.hyracks.dataflow.std.parallel.IHistogram; +import org.apache.hyracks.dataflow.std.parallel.IIterativeHistogram; +import org.apache.hyracks.dataflow.std.parallel.histogram.terneray.SequentialAccessor; +import org.apache.hyracks.dataflow.std.parallel.histogram.terneray.TernaryMemoryTrie; +import org.apache.hyracks.dataflow.std.parallel.util.DualSerialEntry; +import org.apache.hyracks.dataflow.std.parallel.util.HistogramUtils; +import org.apache.hyracks.dataflow.std.parallel.util.ValueSerialEntry; +import org.apache.hyracks.util.string.UTF8StringUtil; + +/** + * @author michael + * Comments: call * + */ +public class TernaryIterativeHistogram implements IIterativeHistogram { + private static final Logger LOGGER = Logger.getLogger(TernaryIterativeHistogram.class.getName()); + + private static final boolean printQuantiles = false; + private final static int DEFAULT_INITIAL_LENGTH = 5; + private final List> stbPrefixies; + private final List> dspPrefixies; + private final List> outPrefixies; + private final boolean fixPointable; + private final double balanceFactor; + private final int outputCards; + + private List> fixPrefixies = null; + private TernaryMemoryTrie tmt; + private boolean iterating = false; + private short prefixLength = 0; + private long totalCount = 0; + private long partialCount = 0; + private final boolean selfGrow; + + /*private int redundant = 0; + private int ommitednt = 0;*/ + + public TernaryIterativeHistogram(int outputCards, double bf, boolean fixPointable, boolean selfGrow) { + this.stbPrefixies = new ArrayList>(); + //For the single value histogram in the future. + this.fixPointable = fixPointable; + this.dspPrefixies = new ArrayList>(); + this.outPrefixies = new ArrayList>(); + this.outputCards = outputCards; + this.balanceFactor = bf; + this.selfGrow = selfGrow; + if (fixPointable) + fixPrefixies = new ArrayList>(); + } + + /* (non-Javadoc) + * @see org.apache.hyracks.dataflow.std.sample.IHistogram#initialize() + */ + @Override + public void initialize() { + if (dspPrefixies.size() > 0) { + iterating = true; + stbPrefixies.clear(); + prefixLength *= 2; + partialCount = 0; + } else { + prefixLength = DEFAULT_INITIAL_LENGTH; + partialCount = 0; + } + if (!selfGrow || tmt == null) + tmt = new TernaryMemoryTrie(prefixLength, selfGrow); + } + + /* (non-Javadoc) + * @see org.apache.hyracks.dataflow.std.sample.IHistogram#getType() + */ + @Override + public FieldType getType() { + // TODO Auto-generated method stub + return FieldType.UTF8; + } + + /* (non-Javadoc) + * @see org.apache.hyracks.dataflow.std.sample.IHistogram#merge(org.apache.hyracks.dataflow.std.sample.IHistogram) + */ + @Override + public void merge(IHistogram ba) throws HyracksDataException { + // TODO Auto-generated method stub + + } + + /* (non-Javadoc) + * @see org.apache.hyracks.dataflow.std.sample.IHistogram#addItem(java.lang.Object) + */ + @Override + public void addItem(UTF8StringPointable item) throws HyracksDataException { + boolean success = false; + if (iterating) { + if (item.getUTF8Length() > prefixLength / 2 + 1) { + StringBuilder sb = new StringBuilder(); + item.toString(sb); + for (int i = 0; i < dspPrefixies.size(); i++) { + if (dspPrefixies.get(i).getKey().equals(sb.substring(0, prefixLength / 2 + 1))) { + success = tmt.insert(new SequentialAccessor(sb.toString()), 0); + if (!success) { + partialCount++; + throw new HyracksDataException(sb.toString() + " length: " + item.getLength() + + " constraint: " + prefixLength); + } + break; + } + } + } + } else { + totalCount++; + StringBuilder sb = new StringBuilder(); + item.toString(sb); + success = tmt.insert(new SequentialAccessor(sb.toString()), 0); + if (!success) + throw new HyracksDataException(item.getLength() + " out of: " + prefixLength); + } + } + + /* (non-Javadoc) + * @see org.apache.hyracks.dataflow.std.sample.IHistogram#countItem(java.lang.Object) + */ + @Override + public void countItem(UTF8StringPointable item) throws HyracksDataException { + // TODO Auto-generated method stub + + } + + /* (non-Javadoc) + * @see org.apache.hyracks.dataflow.std.sample.IHistogram#countReset() + */ + @Override + public void countReset() throws HyracksDataException { + // TODO Auto-generated method stub + + } + + /* (non-Javadoc) + * @see org.apache.hyracks.dataflow.std.sample.IHistogram#getCurrent() + */ + @Override + public int getCurrent() throws HyracksDataException { + // TODO Auto-generated method stub + return 0; + } + + private void orderPrefixies(List> prefixies, boolean compress, boolean isGlobal) + throws HyracksDataException { + List> serialOutput = new ArrayList>(); + for (Entry e : prefixies) + serialOutput.add(new DualSerialEntry(e.getKey(), e.getValue(), false, false)); + Collections.sort(serialOutput); + prefixies.clear(); + if (compress) + compress(serialOutput, prefixies, isGlobal); + else + for (Entry e : serialOutput) + prefixies.add(e); + } + + //Unordered ticks for rangeMap merge + /* (non-Javadoc) + * @see org.apache.hyracks.dataflow.std.sample.IHistogram#generate() + */ + @Override + public List> generate(boolean isGlobal) throws HyracksDataException { + orderPrefixies(outPrefixies, true, isGlobal); + return convertMergeToUTF8(outPrefixies); + } + + private void compress(List> in, List> out, boolean isGlobal) + throws HyracksDataException { + String quantileOut = ""; + double threshold = 0; + int quantiles = 0; + if (isGlobal) { + threshold = totalCount / outputCards; + quantiles = outputCards; + } else { + threshold = (totalCount * balanceFactor) / outputCards; + quantiles = in.size(); + } + int count = 0; + int iPart = 0; + int iCur = 0; + for (int i = 0; i < in.size(); i++) { + if (printQuantiles) { + String sb = in.get(i).getKey(); + quantileOut += sb.toString() + " : " + in.get(i).getValue() + "\n"; + } + if (iPart < quantiles - 1 && (iCur + in.get(i).getValue() / 2) > threshold * (iPart + 1)) { + if (i > 0) + out.add(in.get(i - 1)); + else + out.add(in.get(i)); + out.get(out.size() - 1).setValue(count); + count = 0; + iPart++; + } + count += in.get(i).getValue(); + iCur += in.get(i).getValue(); + } + if (isGlobal && in.size() > 0) { + out.add(in.get(in.size() - 1)); + out.get(out.size() - 1).setValue(count); + } + if (printQuantiles) + LOGGER.info(quantileOut); + quantiles = 0; + count = 0; + for (int i = 0; i < out.size(); i++) + quantiles += out.get(i).getValue(); + for (int i = 0; i < in.size(); i++) + count += in.get(i).getValue(); + LOGGER.info("Before merge: " + in.size() + " After merge: " + out.size() + " on: " + quantiles + " out of " + + count); + } + + private void disperse(String path, boolean deeper) { + tmt.grow(new SequentialAccessor(path), deeper, prefixLength); + } + + /* (non-Javadoc) + * @see org.apache.hyracks.dataflow.std.sample.IIterativeHistogram#disperse(java.lang.Object, int) + */ + @Override + public void disperse() throws HyracksDataException { + // TODO Auto-generated method stub + initialize(); + for (Entry entry : dspPrefixies) { + disperse(entry.getKey(), true); + } + if (fixPointable) { + for (Entry entry : fixPrefixies) + disperse(entry.getKey(), false); + } + } + + private Map updateIteration() throws HyracksDataException { + stbPrefixies.clear(); + tmt.serialize(prefixLength); + Entry entry = null; + Map genExtensible = new HashMap(); + while ((entry = tmt.next(true)) != null) { + stbPrefixies.add(entry); + if (entry.getKey().length() == prefixLength + 1) + genExtensible.put(entry.getKey(), true); + else { + genExtensible.put(entry.getKey(), false); + } + } + return genExtensible; + } + + private void outputPrefix(Map fatherMap, Entry childEntry) { + if (prefixLength == DEFAULT_INITIAL_LENGTH) + return; + int fatherLength = prefixLength / 2; + try { + String key = childEntry.getKey().substring(0, fatherLength + 1); + fatherMap.put(key, fatherMap.get(key) - childEntry.getValue()); + } catch (Exception e) { + e.printStackTrace(); + } + } + + @Override + public boolean needIteration() throws HyracksDataException { + boolean ret = false; + Map genExtensible = updateIteration(); + Map oldPrefixies = new HashMap(); + for (Iterator> itr = dspPrefixies.iterator(); itr.hasNext();) { + Entry entry = itr.next(); + oldPrefixies.put(entry.getKey(), entry.getValue()); + } + dspPrefixies.clear(); + if (stbPrefixies.size() <= 0) { + for (Iterator> itr = oldPrefixies.entrySet().iterator(); itr.hasNext();) { + Entry entry = itr.next(); + if (entry.getValue() > 0) { + disperse(entry.getKey(), false); + outPrefixies.add(entry); + } + } + LOGGER.warning("Double calling iterations without freshing the sampling data"); + return false; + } + for (Entry e : stbPrefixies) { + outputPrefix(oldPrefixies, e); + if (e.getValue() > 1 && e.getValue() > (double) totalCount / outputCards * balanceFactor) { + if (genExtensible.get(e.getKey())) { + dspPrefixies.add(e); + ret = true; + } else { + if (fixPointable) + fixPrefixies.add(e); + else { + disperse(e.getKey(), false); + outPrefixies.add(e); + } + } + } else { + disperse(e.getKey(), false); + outPrefixies.add(e); + } + } + for (Iterator> itr = oldPrefixies.entrySet().iterator(); itr.hasNext();) { + Entry entry = itr.next(); + if (entry.getValue() > 0) { + disperse(entry.getKey(), false); + outPrefixies.add(entry); + } + } + int counto = testCount(outPrefixies); + int countd = testCount(dspPrefixies); + int counta = counto + countd; + int countf = 0; + if (fixPointable) + countf = testCount(fixPrefixies); + LOGGER.info("Counto: " + counto + " countd: " + countd + " counta: " + counta + " countf: " + countf + + " partial: " + partialCount + " length: " + prefixLength + " payload: " + tmt.getPayCount() + + " return: " + ret); + return ret; + } + + private int testCount(List> prefs) { + int count = 0; + for (Entry e : prefs) + count += e.getValue(); + return count; + } + + @Override + public boolean isFixPointable() throws HyracksDataException { + return fixPointable; + } + + @SuppressWarnings("unused") + private List> convertToUTF8(List> prefixies) { + List> output = new ArrayList>(); + for (Entry e : prefixies) { + UTF8StringPointable ustr = (UTF8StringPointable) UTF8StringPointable.FACTORY.createPointable(); + byte[] buf = HistogramUtils.toUTF8Byte(e.getKey()/*.toCharArray()*/, 0); + ustr.set(buf, 0, UTF8StringUtil.getUTFLength(buf, 0)); + output.add(new ValueSerialEntry(ustr, e.getValue())); + } + return output; + } + + private List> convertMergeToUTF8(List> prefixies) { + List> output = new ArrayList>(); + int index = 0; + int accumCount = 0; + String lastKey = ""; + for (Entry e : prefixies) { + if (lastKey.equals(e.getKey())) { + accumCount += e.getValue(); + output.get(index).setValue(accumCount); + } else { + UTF8StringPointable ustr = (UTF8StringPointable) UTF8StringPointable.FACTORY.createPointable(); + byte[] buf = HistogramUtils.toUTF8Byte(e.getKey()/*.toCharArray()*/, 0); + ustr.set(buf, 0, UTF8StringUtil.getUTFLength(buf, 0)); + output.add(new ValueSerialEntry(ustr, e.getValue())); + accumCount = e.getValue(); + lastKey = e.getKey(); + index++; + } + } + return output; + } + + @Override + public List> getFixPointable() throws HyracksDataException { + orderPrefixies(fixPrefixies, false, true); + return convertMergeToUTF8(fixPrefixies); + } +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/terneray/MemoryTernaryNode.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/terneray/MemoryTernaryNode.java new file mode 100644 index 00000000000..93ba77c4217 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/terneray/MemoryTernaryNode.java @@ -0,0 +1,160 @@ +/* + * 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.hyracks.dataflow.std.parallel.histogram.terneray; + +import org.apache.hyracks.dataflow.std.parallel.IStatisticTernaryNode; + +/** + * @author michael + * This node consists of an optional payload field for incremental sequential accessor. + */ +public class MemoryTernaryNode implements IStatisticTernaryNode { + private T payload = null; + private char key = 0; + private IStatisticTernaryNode left = null; + private IStatisticTernaryNode right = null; + private IStatisticTernaryNode middle = null; + private int id = -1; + private boolean grown = false; + //The level limit 16 bits plus the current level 16 bits. + private int level = 0; + private int count = 0; + + public MemoryTernaryNode() { + } + + public MemoryTernaryNode(int level) { + this.level = level; + } + + public MemoryTernaryNode(short limit, short level) { + this.level |= limit << 16; + this.level |= level << 0; + } + + public int getId() { + return id; + } + + public void setId(int id) { + this.id = id; + } + + @Override + public boolean isActive() { + return (getLimit() > getLevel()); + } + + @Override + public char getKey() { + return key; + } + + @Override + public void setKey(char key) { + this.key = key; + } + + @Override + public IStatisticTernaryNode getLeft() { + return left; + } + + @Override + public void setLeft(IStatisticTernaryNode left) { + this.left = left; + } + + @Override + public IStatisticTernaryNode getRight() { + return right; + } + + @Override + public void setRight(IStatisticTernaryNode right) { + this.right = right; + } + + @Override + public IStatisticTernaryNode getMiddle() { + return middle; + } + + @Override + public void setMiddle(IStatisticTernaryNode middle) { + this.middle = middle; + } + + @Override + public short getLimit() { + return (short) (level >> 16); + } + + @Override + public void setLimit(int limit) { + level &= 0xffff; + level |= (limit & 0xffff) << 16; + } + + @Override + public short getLevel() { + return (short) (level & 0xffff); + } + + @Override + public void setLevel(int limit) { + level &= 0xffff0000; + level |= (limit & 0xffff) << 0; + } + + @Override + public int getCount() { + return count; + } + + @Override + public void clearCount() { + this.count = 0; + } + + @Override + public void updateBy(int ub) { + this.count += ub; + } + + @Override + public void setPayload(T payload) { + this.payload = payload; + } + + @Override + public T getPayload() { + return payload; + } + + @Override + public void setGrown() { + this.grown = true; + } + + @Override + public boolean isGrown() { + return grown; + } +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/terneray/SequentialAccessor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/terneray/SequentialAccessor.java new file mode 100644 index 00000000000..86537f49128 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/terneray/SequentialAccessor.java @@ -0,0 +1,99 @@ +/* + * 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.hyracks.dataflow.std.parallel.histogram.terneray; + +import org.apache.hyracks.dataflow.std.parallel.ISequentialAccessor; + +/** + * @author michael + */ +public class SequentialAccessor implements ISequentialAccessor { + boolean initialized = false; + String string; + int cursor = 0; + + public static final SequentialAccessor INSTANCE = new SequentialAccessor(); + + public ISequentialAccessor create(String s) { + return new SequentialAccessor(s); + } + + public SequentialAccessor() { + cursor = 0; + } + + public SequentialAccessor(String str) { + this.string = str; + cursor = 0; + } + + @Override + public char first() { + if (string.length() > 0) { + reset(); + return string.charAt(0); + } else + return 0; + } + + @Override + public char current() { + if (cursor < string.length()) + return string.charAt(cursor); + else + return 0; + } + + @Override + public char next() { + if (++cursor < string.length()) + return string.charAt(cursor); + else { + return 0; + } + } + + @Override + public void reset() { + cursor = 0; + } + + @Override + public char at(int i) { + return string.charAt(i); + } + + @Override + public int length() { + // TODO Auto-generated method stub + return string.length(); + } + + @Override + public int cursor() { + // TODO Auto-generated method stub + return cursor; + } + + @Override + public String toString() { + return string; + } +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/terneray/TernaryMemoryTrie.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/terneray/TernaryMemoryTrie.java new file mode 100644 index 00000000000..73d4f5af13f --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/histogram/terneray/TernaryMemoryTrie.java @@ -0,0 +1,338 @@ +/* + * 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.hyracks.dataflow.std.parallel.histogram.terneray; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map.Entry; +import java.util.logging.Logger; + +import org.apache.hyracks.dataflow.std.parallel.ISequentialAccessor; +import org.apache.hyracks.dataflow.std.parallel.ISequentialTrie; +import org.apache.hyracks.dataflow.std.parallel.IStatisticEntity; +import org.apache.hyracks.dataflow.std.parallel.IStatisticTernaryNode; +import org.apache.hyracks.dataflow.std.parallel.util.DualSerialEntry; + +/** + * @author michael + */ +public class TernaryMemoryTrie implements ISequentialTrie, IStatisticEntity { + @SuppressWarnings("unused") + private static final Logger LOGGER = Logger.getLogger(TernaryMemoryTrie.class.getName()); + private static final short DEFAULT_LEVEL_STEP = 1; + private static final short DEFAULT_ROOT_LEVEL = 0; + private List> serialRoots = null; + private int current = -1; + private final boolean selfGrow; + private int payCount = 0; + private int nodeCount = 0; + private int nodeCreate = 0; + private IStatisticTernaryNode> root = null; + + public int increaseLimit(short limit, short level) { + int newLevel = 0; + newLevel |= (limit + DEFAULT_LEVEL_STEP) << 16; + newLevel |= level; + return newLevel; + } + + public TernaryMemoryTrie() { + this.selfGrow = false; + } + + public TernaryMemoryTrie(short limit, boolean grow) { + this.selfGrow = grow; + root = new MemoryTernaryNode>(limit, DEFAULT_ROOT_LEVEL); + } + + private void pinLoad(IStatisticTernaryNode> node, E p) { + List payload = null; + if ((payload = node.getPayload()) == null) + node.setPayload(payload = new ArrayList()); + payload.add(p); + payCount++; + } + + private boolean insert(E p, int id, IStatisticTernaryNode> node) { + boolean ret = false; + char key; + if (node.equals(root)) + key = p.first(); + else + key = p.current(); + while (p.cursor() < p.length()) { + char nodeKey = node.getKey(); + if (nodeKey == 0) { + ret = true; + node.setKey(key); + } + if (key < node.getKey()) { + IStatisticTernaryNode> left = node.getLeft(); + if (left == null) { + left = new MemoryTernaryNode>(node.getLimit(), node.getLevel()); + nodeCreate++; + node.setLeft(left); + } + node = left; + } else if (key > node.getKey()) { + IStatisticTernaryNode> right = node.getRight(); + if (right == null) { + right = new MemoryTernaryNode>(node.getLimit(), node.getLevel()); + nodeCreate++; + node.setRight(right); + } + node = right; + } else { + node.updateBy(1); + if (p.cursor() + 1 == p.length()) { + node.setId(id); + ret = true; + } else { + IStatisticTernaryNode> mid = node.getMiddle(); + if (null == mid) { + if (node.isActive()) { + mid = new MemoryTernaryNode>(node.getLimit(), (short) (node.getLevel() + 1)); + nodeCreate++; + node.setMiddle(mid); + if (p.length() == p.cursor() + 1) { + mid.setId(id); + mid.updateBy(1); + ret = true; + } + } else { + node.setId(id); + ret = true; + if (selfGrow && p.cursor() == node.getLimit()) + pinLoad(node, p); + break; + } + } + node = mid; + } + key = p.next(); + } + } + if (selfGrow && p.length() == node.getLimit() + 1 && p.current() == p.length()) + pinLoad(node, p); + return ret; + + } + + private IStatisticTernaryNode> traverse(IStatisticTernaryNode> from, E p) { + IStatisticTernaryNode> node = null; + int pos = 0; + while (from != null && pos < p.length()) { + if (p.at(pos) < from.getKey()) + from = from.getLeft(); + else if (p.at(pos) > from.getKey()) + from = from.getRight(); + else { + if (from.getId() != -1) { + node = from; + } + from = from.getMiddle(); + pos++; + } + } + return node; + } + + //Return: true reaches the leaf, false for middle node. + @Override + public boolean insert(E p, int id) { + if (null == root) + root = new MemoryTernaryNode>(DEFAULT_LEVEL_STEP, DEFAULT_ROOT_LEVEL); + return insert(p, id, root); + } + + @Override + public void grow(E p, boolean deeper, short limit) { + if (!selfGrow) + return; + IStatisticTernaryNode> node = traverse(root, p); + if (!deeper) { + List payload = node.getPayload(); + if (payload != null) { + payload.clear(); + node.setPayload(null); + payCount--; + } + } else { + node.setLimit(limit); + node.setGrown(); + Iterator payload = node.getPayload().iterator(); + while (payload.hasNext()) { + insert(payload.next(), 0, node); + } + node.getPayload().clear(); + node.setPayload(null); + payCount--; + } + } + + @Override + public int search(E p) { + if (p.length() < 0) + return -1; + IStatisticTernaryNode> node = traverse(root, p); + if (node == null) + return -1; + else + return node.getId(); + } + + @Override + public boolean delete(E p) { + // TODO Auto-generated method stub + return false; + } + + @Override + public int getCount(E p) { + if (p.length() < 0) + return -1; + IStatisticTernaryNode> node = traverse(root, p); + if (node == null) + return -1; + else + return node.getCount(); + } + + @Override + public void clearCount(E p) { + if (p.length() < 0) + return; + IStatisticTernaryNode> node = traverse(root, p); + if (node != null) + clearChildren(node); + } + + private void clearChildren(IStatisticTernaryNode> node) { + if (node != null) { + node.updateBy(-node.getCount()); + clearChildren(node.getLeft()); + clearChildren(node.getMiddle()); + clearChildren(node.getRight()); + } + } + + @Override + public void updateBy(E p, int ub) { + if (p.length() < 0) + return; + IStatisticTernaryNode> node = traverse(root, p); + if (node != null) + node.updateBy(ub); + } + + private int generateSequences(IStatisticTernaryNode> node, String path, short limit) { + if (node == null) + return 0; + int leftCount = generateSequences(node.getLeft(), path, limit); + String newPath = new String(path); + int childCount = generateSequences(node.getMiddle(), newPath += node.getKey(), limit); + int rightCount = generateSequences(node.getRight(), path, limit); + if (node.getId() != -1) { + short nodeLimit = node.getLimit(); + if (!selfGrow) + serialRoots.add(new DualSerialEntry(path + node.getKey(), + node.getCount() - childCount, false)); + else if (nodeLimit == limit && !node.isGrown()) + serialRoots.add(new DualSerialEntry(path + node.getKey(), + node.getCount() - childCount, false)); + return (node.getCount() + leftCount + rightCount); + } else { + if (node.getMiddle() == null) + return (node.getCount()); + else + return (node.getCount() + rightCount + leftCount); + } + } + + @Override + public void serialize(short limit) { + if (serialRoots == null) + serialRoots = new ArrayList>(); + else + serialRoots.clear(); + String sroot = ""; + generateSequences(root, sroot, limit); + Collections.sort(serialRoots); + current = 0; + } + + @SuppressWarnings("unchecked") + @Override + public E next() { + if (current < serialRoots.size()) { + DualSerialEntry se = serialRoots.get(current++); + return (E) new SequentialAccessor(se.getKey()); + } else + return null; + } + + public Entry next(boolean dbg) { + if (current < serialRoots.size()) { + DualSerialEntry se = serialRoots.get(current++); + return se; + } else + return null; + } + + public int getTotal() { + int count = 0; + for (Entry e : serialRoots) { + count += e.getValue(); + } + return count; + } + + public int getPayCount() { + return payCount; + } + + private void iterativeClean(IStatisticTernaryNode> node) { + if (node == null) + return; + iterativeClean(node.getLeft()); + iterativeClean(node.getMiddle()); + iterativeClean(node.getRight()); + nodeCount++; + if (node.getPayload() != null) + payCount++; + } + + public int verifyClean() { + payCount = 0; + nodeCount = 0; + iterativeClean(root); + return payCount; + } + + public int nodeCount() { + return nodeCount; + } + + public int nodeCreate() { + return nodeCreate; + } +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/sampler/BernoulliSampler.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/sampler/BernoulliSampler.java new file mode 100644 index 00000000000..45591d1dc4a --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/sampler/BernoulliSampler.java @@ -0,0 +1,94 @@ +/* + * 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.hyracks.dataflow.std.parallel.sampler; + +import java.util.Collection; +import java.util.Random; + +public class BernoulliSampler implements ISampler { + private final double percent; + + private final Random rnd; + + private Double nextRnd = null; + + public BernoulliSampler(double percent) { + this.percent = percent / 100; + rnd = new Random(); + } + + public BernoulliSampler(double percent, Random rnd) { + this.percent = percent; + this.rnd = rnd; + stage(); + } + + private void stage() { + nextRnd = rnd.nextDouble(); + } + + private boolean check() { + return nextRnd < percent; + } + + public void setSeed(long seed) { + rnd.setSeed(seed); + } + + @Override + public void sample(T t) { + // TODO Auto-generated method stub + } + + @SuppressWarnings("unchecked") + @Override + public void sample(T... t) { + // TODO Auto-generated method stub + } + + @Override + public Collection getSamples() { + // TODO Auto-generated method stub + return null; + } + + @Override + public int getSize() { + // TODO Auto-generated method stub + return 0; + } + + @Override + public void reset() { + // TODO Auto-generated method stub + } + + @Override + public boolean sampleNext() { + boolean val = check(); + stage(); + return val; + } + + @Override + public void remove() { + // TODO Auto-generated method stub + } +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/sampler/ChainSampler.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/sampler/ChainSampler.java new file mode 100644 index 00000000000..d70a409de30 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/sampler/ChainSampler.java @@ -0,0 +1,107 @@ +/* + * 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.hyracks.dataflow.std.parallel.sampler; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; + +public class ChainSampler implements ISampler { + private int k; + + private int n; + + private long count = 0; + + private int fillSample = 0; + + private List elements; + + private Map replacements; + + private Random random = new Random(); + + public ChainSampler(int k, int n) { + this.k = k; + this.n = n; + elements = new ArrayList(k); + replacements = new HashMap(); + } + + @Override + public void sample(T t) { + int i = (int) (count % n); + if (replacements.containsKey(i)) { + int replace = replacements.get(i); + elements.set(replace, t); + int next = random.nextInt(n); + replacements.remove(i); + replacements.put(next, replace); + } else if (fillSample < k) { + double prob = ((double)Math.min(i, n)) / ((double)n); + if (random.nextDouble() < prob) { + int bucket = fillSample++; + int next = random.nextInt(n); + elements.set(bucket, t); + replacements.put(next, bucket); + } + } + } + + @SuppressWarnings("unchecked") + @Override + public void sample(T... t) { + for (T item : t) + sample(item); + } + + @Override + public Collection getSamples() { + return Collections.unmodifiableCollection(elements); + } + + @Override + public int getSize() { + return (fillSample < k) ? fillSample : k; + } + + @Override + public void reset() { + // TODO Auto-generated method stub + + } + + @Override + public boolean sampleNext() { + // TODO Auto-generated method stub + return false; + } + + @Override + public void remove() { + // TODO Auto-generated method stub + + } + +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/sampler/ISampler.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/sampler/ISampler.java new file mode 100644 index 00000000000..a047f37da8a --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/sampler/ISampler.java @@ -0,0 +1,46 @@ +/* + * 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.hyracks.dataflow.std.parallel.sampler; + +import java.util.Collection; + +public interface ISampler { + public enum SAMPLE_ALGORITHM { + RANDOM, + BERNOULLI, + CHAIN, + RESERVOIR + } + + public void sample(T t); + + @SuppressWarnings("unchecked") + public void sample(T... t); + + public Collection getSamples(); + + public int getSize(); + + public void reset(); + + public boolean sampleNext(); + + public void remove(); +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/sampler/RandomSampler.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/sampler/RandomSampler.java new file mode 100644 index 00000000000..7541248c11d --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/sampler/RandomSampler.java @@ -0,0 +1,102 @@ +/* + * 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.hyracks.dataflow.std.parallel.sampler; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Random; + +public class RandomSampler implements ISampler, Iterator { + + private final List elements = new ArrayList(); + + private int fixCard; + + private Random rand; + + private int curCard; + + public RandomSampler(final Collection base) { + Iterator iter = base.iterator(); + while (iter.hasNext()) + elements.add(iter.next()); + } + + public RandomSampler(final Collection base, int fixCard) { + this(base); + this.fixCard = fixCard; + curCard = fixCard; + rand = new Random(); + } + + @Override + public void sample(T t) { + // TODO Auto-generated method stub + } + + @SuppressWarnings("unchecked") + @Override + public void sample(T... t) { + // TODO Auto-generated method stub + } + + @Override + public Collection getSamples() { + Collection ret = new HashSet(); + while (hasNext()) + ret.add(next()); + return ret; + } + + @Override + public int getSize() { + return curCard; + } + + @Override + public boolean sampleNext() { + // TODO Auto-generated method stub + return false; + } + + @Override + public void remove() { + curCard--; + } + + @Override + public boolean hasNext() { + return (curCard > 0); + } + + @Override + public T next() { + curCard--; + return elements.remove(Math.abs(rand.nextInt()) % elements.size()); + } + + @Override + public void reset() { + curCard = fixCard; + } +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/sampler/ReservoirSampler.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/sampler/ReservoirSampler.java new file mode 100644 index 00000000000..cf0e02064ef --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/sampler/ReservoirSampler.java @@ -0,0 +1,202 @@ +/* + * 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.hyracks.dataflow.std.parallel.sampler; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Random; + +public class ReservoirSampler> implements ISampler { + + public interface IGammaFunction { + enum GAMMA_TYPE { + RANDOM, + ZIPFAN + } + + long apply(long t); + } + + public class SimpleRandom implements IGammaFunction { + + private final Random RAND = new Random(); + + private long n; + + public SimpleRandom(int n) { + super(); + this.n = n; + } + + @Override + public long apply(long t) { + int skipCount = 0; + while (RAND.nextDouble() * t >= n) + skipCount++; + return skipCount; + } + } + + public class SimpleZipfan implements IGammaFunction { + + private Random generator = new Random(); + + private final int n; + + private double w; + + public SimpleZipfan(int n) { + super(); + this.n = n; + this.w = Math.exp(-Math.log(generator.nextDouble()) / n); + } + + @Override + public long apply(long t) { + double term = t - this.n + 1; + double u; + double x; + long gamma; + while (true) { + //generate u and x + u = generator.nextDouble(); + x = t * (this.w - 1.0); + gamma = (long) x; + //test if u <= h(gamma)/cg(x) + double lhs = Math.exp(Math.log(((u * Math.pow(((t + 1) / term), 2)) * (term + gamma)) / (t + x)) + / this.n); + double rhs = (((t + x) / (term + gamma)) * term) / t; + if (lhs < rhs) { + this.w = rhs / lhs; + break; + } + //test if u <= f(gamma)/cg(x) + double y = (((u * (t + 1)) / term) * (t + gamma + 1)) / (t + x); + double denom; + double number_lim; + if (this.n < gamma) { + denom = t; + number_lim = term + gamma; + } else { + denom = t - this.n + gamma; + number_lim = t + 1; + } + + for (long number = t + gamma; number >= number_lim; number--) { + y = (y * number) / denom; + denom = denom - 1; + } + this.w = Math.exp(-Math.log(generator.nextDouble()) / this.n); + if (Math.exp(Math.log(y) / this.n) <= (t + x) / t) { + break; + } + } + return gamma; + } + } + + List elements; + + private int size; + + private boolean orderedExport = false; + + private long skipCount; + + private int currentCount; + + private IGammaFunction skipFunction; + + private IGammaFunction.GAMMA_TYPE type = IGammaFunction.GAMMA_TYPE.ZIPFAN; + + private final Random RANDOM = new Random(); + + public ReservoirSampler(int size) { + elements = new ArrayList(size); + this.size = size; + this.currentCount = 0; + this.skipCount = 0; + if (type.equals(IGammaFunction.GAMMA_TYPE.ZIPFAN)) + this.skipFunction = new SimpleZipfan(size); + else + this.skipFunction = new SimpleRandom(size); + } + + public ReservoirSampler(int size, boolean orderedExport) { + this(size); + this.orderedExport = orderedExport; + } + + @Override + public void sample(T t) { + if (size != elements.size()) { + elements.add(t); + } else { + if (skipCount > 0) { + skipCount--; + } else { + elements.set(RANDOM.nextInt(size), t); + skipCount = skipFunction.apply(currentCount); + } + } + + currentCount++; + } + + @SuppressWarnings("unchecked") + @Override + public void sample(T... t) { + for (T item : t) + sample(item); + } + + @Override + public Collection getSamples() { + if (orderedExport) + Collections.sort(elements); + return Collections.unmodifiableCollection(elements); + } + + @Override + public int getSize() { + return size; + } + + @Override + public void reset() { + // TODO Auto-generated method stub + + } + + @Override + public boolean sampleNext() { + // TODO Auto-generated method stub + return false; + } + + @Override + public void remove() { + // TODO Auto-generated method stub + + } + +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/util/DualSerialEntry.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/util/DualSerialEntry.java new file mode 100644 index 00000000000..5535575093b --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/util/DualSerialEntry.java @@ -0,0 +1,74 @@ +/* + * 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.hyracks.dataflow.std.parallel.util; + +import java.util.Map.Entry; + +public class DualSerialEntry, V extends Comparable> implements Entry, + Comparable> { + private final K key; + private V value; + private boolean obf = true; + private boolean desc = false; + + public DualSerialEntry(K key, V value) { + this.key = key; + this.value = value; + } + + public DualSerialEntry(K key, V value, boolean obf) { + this(key, value); + this.obf = obf; + } + + public DualSerialEntry(K key, V value, boolean obf, boolean desc) { + this(key, value, obf); + this.desc = desc; + } + + @Override + public K getKey() { + return key; + } + + @Override + public V getValue() { + return value; + } + + @Override + public V setValue(V value) { + return this.value = value; + } + + @Override + public int compareTo(DualSerialEntry o) { + if (obf) { + if (desc) + return (o.getValue().compareTo(value)); + else + return (value.compareTo(o.getValue())); + } else { + if (desc) + return (o.getKey().compareTo(key)); + else + return (key.compareTo(o.getKey())); + } + } +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/util/HistogramUtils.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/util/HistogramUtils.java new file mode 100644 index 00000000000..3334a7b7665 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/util/HistogramUtils.java @@ -0,0 +1,203 @@ +/* + * 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.hyracks.dataflow.std.parallel.util; + +import org.apache.hyracks.api.exceptions.HyracksDataException; +import org.apache.hyracks.data.std.api.IPointable; +import org.apache.hyracks.data.std.primitive.DoublePointable; +import org.apache.hyracks.data.std.primitive.FloatPointable; +import org.apache.hyracks.data.std.primitive.IntegerPointable; +import org.apache.hyracks.data.std.primitive.LongPointable; +import org.apache.hyracks.data.std.primitive.ShortPointable; +import org.apache.hyracks.data.std.primitive.UTF8StringPointable; +import org.apache.hyracks.util.string.UTF8StringUtil; + +/** + * @author michael + */ +public class HistogramUtils { + private static final boolean USE_SOFT = false; + + public static byte[] ansiToUTF8Byte(String ansiStr, int start) { + return UTF8StringUtil.writeStringToBytes(ansiStr); + } + + public static byte[] toUTF8Byte(String str, int start) { + return UTF8StringUtil.writeStringToBytes(str); + } + + public static byte ansiByteAt(byte[] b, int s) throws HyracksDataException { + int c = b[s] & 0xff; + switch (c >> 4) { + case 0: + case 1: + case 2: + case 3: + case 4: + case 5: + case 6: + case 7: + return b[s]; + case 12: + case 13: + throw new HyracksDataException( + "Binary exception: Current streaming histogram supports ansi string only."); + case 14: + throw new HyracksDataException( + "Triple exception: Current streaming histogram supports ansi string only."); + default: + throw new IllegalArgumentException(); + } + } + + public static long ansiMappingToLong(UTF8StringPointable uStr, int s, int len) throws HyracksDataException { + long lenToLong = 0; + int cStart = uStr.getCharStartOffset(); + int nChars = uStr.getUTF8Length(); + if (len > 9) + throw new HyracksDataException( + "Length exception: Current streaming histogram support nine characters at most"); + for (int i = 0; i < s + len; i++) { + char c = 0; + if (i < nChars) + c = uStr.charAt(cStart); + else + break; + cStart += UTF8StringUtil.getModifiedUTF8Len(c); + //Currently, the streaming histogram support ansi string only, the exception will be thrown otherwise. + if (i < s) + continue; + lenToLong |= ((long) (c - 32)) << ((len - i + s - 1) * 7); + } + return lenToLong; + } + + public static UTF8StringPointable longMappingToAnsiStrict(long quantile, int len) { + UTF8StringPointable uStr = new UTF8StringPointable(); + byte[] uByte = new byte[len + 1]; + for (int i = 0; i < len; i++) { + byte b = (byte) ((((quantile) >> i * 7) & 0x7f) + 32); + if (b < 0) + b = 0; + uByte[len - i/* + 1*/] = b; + } + //uByte[0] = (byte) (len << 16 >> 24); + uByte[0] = (byte) (len & 0xff); + uStr.set(uByte, 0, len + 1); + return uStr; + } + + //Continuously reverting the string and skip the illegal range of UTF8 chars. + public static UTF8StringPointable longMappingToAnsiSoft(long quantile, int len) { + UTF8StringPointable uStr = new UTF8StringPointable(); + byte[] uByte = new byte[len * 3 + 1]; + for (int i = 0; i < len; i++) { + byte b = (byte) ((((quantile) >> i * 7) & 0x7f) + 32); + if (b < 0) { + byte[] bs = new byte[3]; + bs[0] = b; + char c = UTF8StringUtil.charAt(bs, 0); + int l = UTF8StringUtil.getModifiedUTF8Len(c); + System.arraycopy(bs, 0, uByte, len - i + 1, l); + len += (l - 1); + } else + uByte[len - i/* + 1*/] = b; + } + //uByte[0] = (byte) (len << 16 >> 24); + uByte[0] = (byte) (len & 0xff); + uStr.set(uByte, 0, len + 1); + return uStr; + } + + public static UTF8StringPointable longMappingToAnsi(long quantile, int len) { + if (USE_SOFT) + return longMappingToAnsiSoft(quantile, len); + else + return longMappingToAnsiStrict(quantile, len); + } + + public static double ansiMappingToQuantile(IPointable uStr, int s, int len) throws HyracksDataException { + return (double) ansiMappingToLong((UTF8StringPointable) uStr, s, len); + } + + public static IPointable quantileRevertToAnsi(double quantile, int len) { + return longMappingToAnsi((long) quantile, len); + } + + public static double integerMappingToQuantile(IPointable ip) { + return (double) ((IntegerPointable) ip).getInteger(); + } + + public static IPointable quantileRevertToInteger(double d) { + IntegerPointable ip = (IntegerPointable) IntegerPointable.FACTORY.createPointable(); + byte[] buf = new byte[IntegerPointable.TYPE_TRAITS.getFixedLength()]; + ip.set(buf, 0, IntegerPointable.TYPE_TRAITS.getFixedLength()); + ip.setInteger((int) d); + return ip; + } + + public static double longMappingToQuantile(IPointable lp) { + return (double) ((LongPointable) lp).getLong(); + } + + public static IPointable quantileRevertToLong(double d) { + LongPointable lp = (LongPointable) LongPointable.FACTORY.createPointable(); + byte[] buf = new byte[LongPointable.TYPE_TRAITS.getFixedLength()]; + lp.set(buf, 0, LongPointable.TYPE_TRAITS.getFixedLength()); + lp.setLong((long) d); + return lp; + } + + public static double doubleMappingToQuantile(IPointable dp) { + return (double) ((DoublePointable) dp).getDouble(); + } + + public static IPointable quantileRevertToDouble(double d) { + DoublePointable dp = (DoublePointable) DoublePointable.FACTORY.createPointable(); + byte[] buf = new byte[DoublePointable.TYPE_TRAITS.getFixedLength()]; + dp.set(buf, 0, DoublePointable.TYPE_TRAITS.getFixedLength()); + dp.setDouble(d); + return dp; + } + + public static double shortMappingToQuantile(IPointable sp) { + return (double) ((ShortPointable) sp).getShort(); + } + + public static IPointable quantileRevertToShort(double d) { + ShortPointable sp = (ShortPointable) ShortPointable.FACTORY.createPointable(); + byte[] buf = new byte[ShortPointable.TYPE_TRAITS.getFixedLength()]; + sp.set(buf, 0, ShortPointable.TYPE_TRAITS.getFixedLength()); + sp.setShort((short) d); + return sp; + } + + public static double floatMappingToQuantile(IPointable fp) { + return (double) ((FloatPointable) fp).getFloat(); + } + + public static IPointable quantileRevertToFloat(double d) { + FloatPointable fp = (FloatPointable) FloatPointable.FACTORY.createPointable(); + byte[] buf = new byte[FloatPointable.TYPE_TRAITS.getFixedLength()]; + fp.set(buf, 0, FloatPointable.TYPE_TRAITS.getFixedLength()); + fp.setFloat((float) d); + return fp; + } +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/util/ValueSerialEntry.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/util/ValueSerialEntry.java new file mode 100644 index 00000000000..a8f606f3862 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/parallel/util/ValueSerialEntry.java @@ -0,0 +1,56 @@ +/* + * 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.hyracks.dataflow.std.parallel.util; + +import java.util.Map.Entry; + +public class ValueSerialEntry> implements Entry, + Comparable> { + private final K key; + private V value; + private final boolean asc = false; + + public ValueSerialEntry(K key, V value) { + this.key = key; + this.value = value; + } + + @Override + public K getKey() { + return key; + } + + @Override + public V getValue() { + return value; + } + + @Override + public V setValue(V value) { + return this.value = value; + } + + @Override + public int compareTo(ValueSerialEntry o) { + if (asc) + return (o.getValue().compareTo(value)); + else + return (value.compareTo(o.getValue())); + } +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/parallel/TrieTests.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/parallel/TrieTests.java new file mode 100644 index 00000000000..425dda8655a --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/parallel/TrieTests.java @@ -0,0 +1,115 @@ +/* + * 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.hyracks.dataflow.std.parallel; + +import java.io.BufferedReader; +import java.io.ByteArrayOutputStream; +import java.io.FileReader; +import java.io.PrintStream; +import java.util.Map.Entry; +import java.util.logging.Logger; + +import org.apache.hyracks.data.std.primitive.UTF8StringPointable; +import org.apache.hyracks.dataflow.std.parallel.histogram.terneray.SequentialAccessor; +import org.apache.hyracks.dataflow.std.parallel.histogram.terneray.TernaryMemoryTrie; +import org.apache.hyracks.dataflow.std.parallel.util.HistogramUtils; +import org.junit.Before; +import org.junit.Test; +import org.apache.hyracks.util.string.UTF8StringUtil; + +import junit.framework.TestCase; + +/** + * @author michael + */ +public class TrieTests extends TestCase { + private static final Logger LOGGER = Logger.getLogger(TrieTests.class.getName()); + private final ByteArrayOutputStream outContent = new ByteArrayOutputStream(); + private final ByteArrayOutputStream errContent = new ByteArrayOutputStream(); + private final static int DEFAULT_COLUMN = 5; + private final static int ADDRESS_COLUMN = 2; + private final static int COMMENT_COLUMN = 8; + private final static int REGION_COLUMN = 1; + private final static int ZIPFAN_COLUMN = 0; + private static final short DEFAULT_TRIE_LIMIT = 2; + private static final boolean DEFAULT_SELF_GROW = true; + + @Before + public void setUpStreams() { + System.setOut(new PrintStream(outContent)); + System.setErr(new PrintStream(errContent)); + } + + @Test + public void testTernaryTrie() throws Exception { + TernaryMemoryTrie tmt = new TernaryMemoryTrie(DEFAULT_TRIE_LIMIT, + DEFAULT_SELF_GROW); + BufferedReader br = new BufferedReader(new FileReader("data/tpch0.001/orders.tbl")); + String line = null; + while (null != (line = br.readLine())) { + String[] fields = line.split("\\|"); + UTF8StringPointable key = (UTF8StringPointable) UTF8StringPointable.FACTORY.createPointable(); + String strD = fields[COMMENT_COLUMN]; + byte[] buf = HistogramUtils.toUTF8Byte(strD/*.toCharArray()*/, 0); + key.set(buf, 0, UTF8StringUtil.getUTFLength(buf, 0)); + StringBuilder sb = new StringBuilder(); + UTF8StringUtil.toString(sb, key.getByteArray(), 0); + SequentialAccessor sa = new SequentialAccessor(sb.toString()); + tmt.insert(sa, 0); + } + tmt.serialize(DEFAULT_TRIE_LIMIT); + int count = tmt.getTotal(); + LOGGER.info("Total: " + count); + //SequentialAccessor sa = null; + Entry si = null; + int total = 0; + String quantileOut = ""; + while ((si = tmt.next(true)) != null) { + quantileOut += (si.getKey() + ", " + si.getValue() + "\n"); + total += si.getValue(); + } + LOGGER.info(quantileOut); + LOGGER.info("post total: " + total); + /*List> quantiles = tmt.generate(); + for (int i = 0; i < quantiles.size(); i++) { + StringBuilder sb = new StringBuilder(); + UTF8StringPointable.toString(sb, quantiles.get(i).getKey().getByteArray(), 0); + System.out.print("<" + sb.toString() + ", " + quantiles.get(i).getValue() + ">\n"); + } + br.close(); + tmt.countReset(); + System.out.println("Verification"); + br = new BufferedReader(new FileReader("/Users/michael/Desktop/tpch_2_16_1/dbgen/orders.tbl")); + line = null; + while (null != (line = br.readLine())) { + String[] fields = line.split("\\|"); + UTF8StringPointable key = (UTF8StringPointable) UTF8StringPointable.FACTORY.createPointable(); + String strD = fields[COMMENT_COLUMN]; + byte[] buf = SampleUtils.toUTF8Byte(strD.toCharArray(), 0); + key.set(buf, 0, UTF8StringPointable.getUTFLength(buf, 0)); + dth.countItem(key); + } + quantiles = dth.generate(); + for (int i = 0; i < quantiles.size(); i++) { + StringBuilder sb = new StringBuilder(); + UTF8StringPointable.toString(sb, quantiles.get(i).getKey().getByteArray(), 0); + System.out.print("<" + i + ", " + sb.toString() + ", " + quantiles.get(i).getValue() + ">\n"); + }*/ + } +} diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/parallel/TypeTranslateTest.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/parallel/TypeTranslateTest.java new file mode 100644 index 00000000000..4a1fc35faec --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/test/java/org/apache/hyracks/dataflow/std/parallel/TypeTranslateTest.java @@ -0,0 +1,84 @@ +/* + * 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.hyracks.dataflow.std.parallel; + +import static org.junit.Assert.assertEquals; + +import java.util.logging.Logger; + +import org.apache.hyracks.api.exceptions.HyracksException; +import org.apache.hyracks.data.std.api.IPointable; +import org.apache.hyracks.data.std.primitive.IntegerPointable; +import org.apache.hyracks.data.std.primitive.UTF8StringPointable; +import org.apache.hyracks.dataflow.std.parallel.util.HistogramUtils; +import org.apache.hyracks.util.string.UTF8StringUtil; +import org.junit.Test; + +public class TypeTranslateTest { + private static final Logger LOGGER = Logger.getLogger(TypeTranslateTest.class.getName()); + + @Test + public void testIntPoitable() throws HyracksException { + //IBinaryComparator comp = PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY).createBinaryComparator(); + byte[] ip255 = new byte[4]; + ip255[3] |= 0xff; + IPointable ip = new IntegerPointable(); + ip.set(ip255, 0, IntegerPointable.TYPE_TRAITS.getFixedLength()); + int iip = ((IntegerPointable) ip).getInteger(); + assertEquals(iip, 255); + ip255[2] |= 0xff; + ip.set(ip255, 0, IntegerPointable.TYPE_TRAITS.getFixedLength()); + iip = ((IntegerPointable) ip).getInteger(); + assertEquals(iip, 65535); + + String longString = new String("2345n,3+)*(&)*&)**UPIUIPPIJKLH7youihuh1kjerhto34545435" + + "8t73048534j5;kj;krejtpreiutpiq34n;krnq;kwerj;qwkj4pi32ou4j;wker;qwernqwe/mr" + + "nqwlh432j423nn4.qmrnqm.wn34lj23q4.q3nw4.mqn4lhq2j34n3qmn4.w34hnjqk2n4.3mn4." + + "3wqnr.mqweh4\"\"[][]][]<>()j3qn.4mqnw34hqjkw4nmersnhjknwemrw.r中h23nwrjjjkh5"); + IPointable sp = new UTF8StringPointable(); + byte[] bsc = HistogramUtils.ansiToUTF8Byte(longString, 0); + sp.set(bsc, 0, bsc.length); + LOGGER.info("The pointable string has the length: " + sp.getLength() + " from origin length: " + + longString.length()); + StringBuilder sb = new StringBuilder(); + UTF8StringUtil.toString(sb, sp.getByteArray(), 0); + String s1 = sb.toString(); + String s2 = longString; + assertEquals(s1, s2); + + IPointable usp = new UTF8StringPointable(); + byte[] ubsc = HistogramUtils.toUTF8Byte(new String("横空出世"), 0); + usp.set(ubsc, 0, ubsc.length); + StringBuilder usb = new StringBuilder(); + UTF8StringUtil.toString(usb, usp.getByteArray(), 0); + String us1 = usb.toString(); + String us2 = "横空出世"; + assertEquals(us1, us2); + + long quantile = HistogramUtils.ansiMappingToLong((UTF8StringPointable) sp, 0, 8); + UTF8StringPointable sQuan = HistogramUtils.longMappingToAnsi(quantile, 8); + StringBuilder sb1 = new StringBuilder(); + UTF8StringUtil.toString(sb1, sQuan.getByteArray(), 0); + assertEquals(sb.toString().substring(0, 8), sb1.toString()); + LOGGER.info("The origin length is: " + s2.length() + " of the string: " + s2 + " by atomically cut as: " + + sb1.length() + " of the string " + sb1); + return; + } +} diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/data/skew/zipfan.tbl b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/data/skew/zipfan.tbl new file mode 100644 index 00000000000..c9bbd1e3d56 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/data/skew/zipfan.tbl @@ -0,0 +1,10000 @@ +6.103292690632419E7 0 +4.597354906360063E7 1 +1.2202737766500251E7 2 +2.768255225636733E7 3 +2.1273498919091545E7 4 +2.1273498658211585E7 5 +2.768255223855939E7 6 +2.1273499630639233E7 7 +2.7682554127548236E7 8 +4.5973549657646984E7 9 +37.3474985396657 10 +1.2202735553967604E7 11 +5.1143521257639274E7 12 +4.5973549217406556E7 13 +5.7599627462235905E7 14 +3.7090212266472675E7 15 +4.5973550290043905E7 16 +2.127349885220329E7 17 +1.2202733183475034E7 18 +5.11435306433012E7 19 +6.103292773242727E7 20 +4.5973548978598125E7 21 +6.103292787567197E7 22 +31.202979698994184 23 +1.2202734169790886E7 24 +9719023.24241583 25 +4.5973548993541986E7 26 +3.709021555502044E7 27 +5.114352186350481E7 28 +4.5973549187567346E7 29 +4.597354922176383E7 30 +6.698296213318033 31 +13.857449987990316 32 +2.127349877204466E7 33 +2.127350421901793E7 34 +6.524698432916729 35 +4.597355102467164E7 36 +2.7682551999287408E7 37 +4.597354946737119E7 38 +1.220273311154998E7 39 +5.114352273585493E7 40 +6.103292650523152E7 41 +2.1273498342354327E7 42 +2.7682552575483803E7 43 +1.2202733585152576E7 44 +5.75996271980796E7 45 +6.103292740421844E7 46 +5.1643161586800375 47 +11.507688912062633 48 +2.127349943481229E7 49 +4.597354952437728E7 50 +3.709021221614585E7 51 +2.7682552153728165E7 52 +2.7682551878780656E7 53 +9719023.008824056 54 +178.0254453266494 55 +4.597354893410924E7 56 +2.7682551602498725E7 57 +5.7599627079399265E7 58 +9719023.049460392 59 +5.114352091648796E7 60 +6.103292653810832E7 61 +5.759962722323264E7 62 +1.2202733481062122E7 63 +3.709027226892748E7 64 +9719023.46640378 65 +9719023.888621178 66 +5.114353512270852E7 67 +9719023.091550823 68 +9719023.1130311 69 +2.768255254041544E7 70 +4.5973549313294984E7 71 +9719023.169055622 72 +122.39605006404459 73 +5.114352620586449E7 74 +4.597354919698816E7 75 +2.7682552047322378E7 76 +5.759962892758276E7 77 +4.597355011631574E7 78 +5.759962735140505E7 79 +5.1143520955129914E7 80 +5.7599627951847576E7 81 +4.597354956847042E7 82 +2.127349839839174E7 83 +9719023.198993744 84 +2.1273498794406414E7 85 +5.759962732131344E7 86 +2.1273498816597164E7 87 +2.1273500875360303E7 88 +6.932274197603845 89 +9719023.046567297 90 +2.768255197245426E7 91 +2.7682552409675602E7 92 +3.709021575416671E7 93 +5.759962745706978E7 94 +2.1273498751980025E7 95 +5.114352237267396E7 96 +3.709021221483229E7 97 +3.709021493776258E7 98 +5.759962737470721E7 99 +9719023.035220155 100 +14.73913740041341 101 +6.103292750474899E7 102 +1.2202733265517166E7 103 +1.2202734984941069E7 104 +4.597354881406735E7 105 +5.438728038085896 106 +9719023.247149616 107 +3.709021323930741E7 108 +2.7682552393061806E7 109 +5.114352129095241E7 110 +1.2202733698458744E7 111 +5.759962704648505E7 112 +3.7090212694018506E7 113 +5.7599638313194826E7 114 +5.114352049690706E7 115 +23.144589569905108 116 +419.4544987800867 117 +1.2202735005100189E7 118 +5.7599627562948845E7 119 +1.220273440242061E7 120 +4.597354893278423E7 121 +4.0768564617501175 122 +3.709021299121563E7 123 +6.1032926927092575E7 124 +5.759962711113698E7 125 +3.709021385178038E7 126 +9719023.762234181 127 +2.1273498828093924E7 128 +3.709021715396273E7 129 +3.7090213689079374E7 130 +6.103292926556671E7 131 +1.220273431668202E7 132 +5.759962723264669E7 133 +2.768255189748913E7 134 +9719024.40362446 135 +1.2202733027736485E7 136 +9719023.103514075 137 +5.759962782709858E7 138 +1.2202733345597802E7 139 +1.2202735647398967E7 140 +3.709022049952629E7 141 +9719023.943536077 142 +6.103292751210822E7 143 +5.759962703039571E7 144 +4.597354939907181E7 145 +21.97917519097534 146 +3.709021258329708E7 147 +10.423379996733855 148 +9.91331713716551 149 +6.1032926758376785E7 150 +9719023.216731178 151 +5.759962773301375E7 152 +4.311796063226326 153 +5.114352128109139E7 154 +2.768255217239296E7 155 +5.7599627127249695E7 156 +6.1032926685623206E7 157 +2.768255246374804E7 158 +6.103292722103444E7 159 +2.127349853581577E7 160 +1.2202734346386092E7 161 +6.103292696969099E7 162 +5.759963058372386E7 163 +9719023.508731471 164 +2.1273498958331022E7 165 +6.103294989387998E7 166 +1.2202733253475444E7 167 +3.709021303529015E7 168 +2.1273500963897247E7 169 +47.171214385024385 170 +9719023.223457834 171 +5.759962755779424E7 172 +4.159748630362639 173 +12.09507530035033 174 +3.709021687879332E7 175 +5.759962708671397E7 176 +1.2202733328308484E7 177 +5.114352138710155E7 178 +5.114352055086331E7 179 +6.1032927560505204E7 180 +9719023.076454042 181 +3.709021293998515E7 182 +6.10329265992192E7 183 +2.76825521982148E7 184 +4.597354909969836E7 185 +1.2202732943592615E7 186 +5.1143521764736146E7 187 +3.7090217202722244E7 188 +4.5973551918044016E7 189 +4.597354918174158E7 190 +6.103292725057487E7 191 +5.759962723665905E7 192 +5.759962827076519E7 193 +11.829309829497465 194 +9719023.189139873 195 +4.597354884087484E7 196 +4.754282883986795 197 +6.103292815043829E7 198 +9719023.367854903 199 +1.2202733601628367E7 200 +3.152955886781862 201 +4.597354889602031E7 202 +6.103293118888204E7 203 +9719023.227444548 204 +4.108491225701911 205 +3.709021522298089E7 206 +1.22027382359241E7 207 +4.531270070327019 208 +3.709021267677188E7 209 +5.759963069410597E7 210 +9719023.048727797 211 +6.10329303261668E7 212 +1.2202736080460163E7 213 +2.7682555442420244E7 214 +1.220273541259847E7 215 +9719023.80265805 216 +2.768255155305657E7 217 +2.768255325659347E7 218 +3.7090213007545285E7 219 +9719023.123663712 220 +2.127349871634925E7 221 +1.2202737073696502E7 222 +2.127349874460145E7 223 +2.127349904204756E7 224 +3.709021269345328E7 225 +5.11435206992308E7 226 +2.768255377943476E7 227 +1.2202733044810945E7 228 +2.1273500168318704E7 229 +6.103292717261277E7 230 +1.220273813639442E7 231 +3.709021236505535E7 232 +5.7599627098371014E7 233 +1.2202733665080726E7 234 +4.597354941393182E7 235 +179.96921114843428 236 +5.75996273323657E7 237 +2.127354282979216E7 238 +2.768255276741093E7 239 +5.114352108575327E7 240 +2.1273499041693468E7 241 +5.1143520532074034E7 242 +3.709021248421214E7 243 +2.7682552850727346E7 244 +3.709021259336548E7 245 +13.662674244487057 246 +1.220273380923339E7 247 +5.759962734401284E7 248 +9719024.285415068 249 +5.11435211462837E7 250 +5.7599627320866145E7 251 +21.500065356854336 252 +5.114352108222977E7 253 +7.088179617150363 254 +9719023.248036698 255 +2.127349841003122E7 256 +9719023.093235694 257 +4.597354944693067E7 258 +1.220273365431873E7 259 +1.2202733460971348E7 260 +2.768255159262766E7 261 +1.2202736662659619E7 262 +6.103292683615028E7 263 +1.220273368215553E7 264 +2.7682552544038557E7 265 +4.597354913887314E7 266 +2.768255659576334E7 267 +5.1143525334848754E7 268 +9719023.498239907 269 +5.1143520545096904E7 270 +3.7090213710951395E7 271 +9719023.777943963 272 +1.2202732880460955E7 273 +4.597354893991944E7 274 +2.768255167017046E7 275 +2.1273498594807446E7 276 +9719023.254746618 277 +2.1273499004097402E7 278 +3.709021261954107E7 279 +1.2202733741684794E7 280 +5.114352086600072E7 281 +5.1143520779586755E7 282 +5.7599627208167166E7 283 +9719023.021286823 284 +13.622834805485654 285 +5.7599627284503594E7 286 +1.220273347616597E7 287 +1.220273354180764E7 288 +2.76825519537288E7 289 +9719023.570168879 290 +2.1273499240915857E7 291 +5.759963167678368E7 292 +2.127349902202498E7 293 +15.502781333717305 294 +1.22027337161158E7 295 +4.597355577757419E7 296 +4.597354886448575E7 297 +6.103293112436113E7 298 +2.127349858749005E7 299 +9719023.154664662 300 +6.103292732832657E7 301 +8.208645966780923 302 +5.114352163158718E7 303 +4.5973554471551694E7 304 +9719023.173330545 305 +3.31367498728442 306 +4.59735493355912E7 307 +5.1143520516873196E7 308 +2.1273498755451795E7 309 +1.2202733641168678E7 310 +4.5973550012838945E7 311 +5.759962711199751E7 312 +9719023.198507566 313 +9719023.23844906 314 +37.646411445286276 315 +45.52589139919693 316 +5.114352353451484E7 317 +4.597355956176297E7 318 +1.2202734461172752E7 319 +2.1273498439153206E7 320 +9719023.102849253 321 +896.700029298464 322 +9719023.306705527 323 +2.1273498528122783E7 324 +16.101984108297973 325 +4.597355009645373E7 326 +5.759962744671697E7 327 +3.709021226703021E7 328 +2.1273499296957042E7 329 +9719023.692034526 330 +5.11435204420698E7 331 +6.1032927003135756E7 332 +3.7090212631500416E7 333 +21.922644482361203 334 +3.7090215473558106E7 335 +5.114352057125507E7 336 +13.798448798937548 337 +3.709021321273632E7 338 +2.1273499657694697E7 339 +23.16785378733547 340 +4.5973549048880376E7 341 +2.768255158687148E7 342 +2.1273499308526598E7 343 +26.09021579396048 344 +4.597354895152025E7 345 +9719023.614682045 346 +6.1032926530998826E7 347 +5.114352084862867E7 348 +2.7682551991223782E7 349 +3.709021361834817E7 350 +9719023.652597787 351 +1.2202733459550852E7 352 +2.127349866653485E7 353 +1.220273386903716E7 354 +3.709021303298497E7 355 +1.2202734057572206E7 356 +9719023.842814157 357 +36.73304927126949 358 +5.75996271468612E7 359 +4.597354879601434E7 360 +2.12734983681706E7 361 +9719023.264703011 362 +14.51795513217107 363 +5.7599627149736606E7 364 +1.2202733935522083E7 365 +1.2202733426790092E7 366 +2.7682552132514916E7 367 +6.840956534833044 368 +9719023.144549318 369 +2.7682554323772855E7 370 +2.12734983235603E7 371 +5.114352126032728E7 372 +6.1032926794028886E7 373 +9719023.477473248 374 +2.127349905036106E7 375 +1.220273301437362E7 376 +4.5973549421074174E7 377 +4.59735490478663E7 378 +5.114352150948288E7 379 +6.103292703975711E7 380 +2.1273498318018027E7 381 +9719023.33747008 382 +4.214060161810848 383 +5.7599627076007895E7 384 +1.2202733282897862E7 385 +2.7682552809332915E7 386 +9719023.000077548 387 +2.1273499502279498E7 388 +4.597354886899731E7 389 +6.103292698319727E7 390 +5.759962713177846E7 391 +2.127349876852373E7 392 +6.103292934635828E7 393 +4.597354968434732E7 394 +2.127349853739007E7 395 +6.103293756666699E7 396 +9719023.212577038 397 +5.759962708063973E7 398 +1.2202732946579622E7 399 +9719023.309473999 400 +5.7599627473187715E7 401 +12.170365909914452 402 +2.127349863696417E7 403 +2.768255195522852E7 404 +9719023.149173822 405 +5.7599627733652025E7 406 +9719023.261773352 407 +4.597354925216135E7 408 +5.1143520550194904E7 409 +3.709021298996656E7 410 +2.127349909403168E7 411 +6.034240238863432 412 +6.103293074767974E7 413 +2.127349837716397E7 414 +4.597354892598266E7 415 +5.114352037669203E7 416 +1.2202733704393957E7 417 +5.811711573939856 418 +5.759962729092476E7 419 +4.5973549923061185E7 420 +2.1273498683794186E7 421 +4.597354908456545E7 422 +4.597354890826633E7 423 +4.5973552973566286E7 424 +6.103292928798822E7 425 +3.709021714979197E7 426 +9719023.060311416 427 +5.114352200809999E7 428 +89.69681789781191 429 +2.127349863978614E7 430 +4.597354892245343E7 431 +27.770508669981503 432 +3.709021263732269E7 433 +23.443371150939573 434 +5.1143520856561475E7 435 +9719023.009889271 436 +19.374181737854702 437 +2.7682552397266544E7 438 +5.1143521214924656E7 439 +6.10329273700286E7 440 +3.709021239058778E7 441 +2.7682555511942543E7 442 +6.103292735675068E7 443 +4.597354929371584E7 444 +5.114352125941944E7 445 +1.2202732885316702E7 446 +2.7682552087579913E7 447 +17.5674598561 448 +6.103292823963176E7 449 +15.757255886515141 450 +5.759962922982991E7 451 +6.103292707129065E7 452 +9719023.011511674 453 +2.768255188608768E7 454 +9719023.051144548 455 +4.597354944239677E7 456 +9719023.280494224 457 +5.759962819107953E7 458 +42.63932496038518 459 +44.23934519075901 460 +4.597354895817137E7 461 +2.7682551589550596E7 462 +3.709022496001259E7 463 +26.24580343191897 464 +4.597354973475028E7 465 +5.1143525846639E7 466 +3.709021328747236E7 467 +9719023.34320727 468 +4.597354891506712E7 469 +18.137716044026618 470 +2.7682554842215694E7 471 +9719023.664149895 472 +5.759962728147252E7 473 +2.1273499181335565E7 474 +2.127349843114982E7 475 +1.220273355943418E7 476 +1.2202735785843736E7 477 +4.597354945991451E7 478 +6.1032927118303485E7 479 +5.114352071042984E7 480 +6.1032927415522605E7 481 +6.10329277012148E7 482 +5.759962725656294E7 483 +5.759962705687458E7 484 +9719023.998384666 485 +2.7682553019257266E7 486 +1.220273287516216E7 487 +1.2202733774783272E7 488 +5.114352085274081E7 489 +9719024.84472207 490 +1.2202733427668829E7 491 +9719024.08029044 492 +5.759962713274733E7 493 +1074.3583577749591 494 +9719023.189271461 495 +2.768255153142511E7 496 +5.1143520947629794E7 497 +9719023.635439666 498 +5.759962709187385E7 499 +14.82926565176361 500 +5.114352051082522E7 501 +4.597355193226669E7 502 +2.7682556356878776E7 503 +2.7682551974245906E7 504 +5.114352097671984E7 505 +1.2202733551845586E7 506 +3.709021266820107E7 507 +4.5973548917583615E7 508 +6.103292807898789E7 509 +1.2202733560563806E7 510 +9.043020423405782 511 +5.114352073024576E7 512 +8.16861323315278 513 +12.296164788736911 514 +2.1273499227081634E7 515 +6.10329265936171E7 516 +5.114352086698208E7 517 +6.103293046083221E7 518 +2.7682553504089486E7 519 +24.86287219085258 520 +2.768255175434981E7 521 +3.709021205518209E7 522 +6.103293331736334E7 523 +3.709021318332062E7 524 +170.82339574568294 525 +1.2202734317333957E7 526 +6.103292688016652E7 527 +3.709021686895241E7 528 +1.2202736256279988E7 529 +2.127349838221085E7 530 +4.597354934127004E7 531 +3.709021213811628E7 532 +19.84723733952607 533 +13.609468182328913 534 +3.709021233816448E7 535 +3.709021231190346E7 536 +6.103292683928329E7 537 +6.103292681967158E7 538 +5.1143523757821634E7 539 +5.759962862029663E7 540 +9719023.089038573 541 +10.03667480154962 542 +3.709021219075805E7 543 +2.7682552118457846E7 544 +9719037.830525108 545 +5.1143522927234754E7 546 +5.759962750467917E7 547 +2.7682552258023843E7 548 +33.08037434990339 549 +5.114352641495022E7 550 +4.597355581609311E7 551 +1.22027328934518E7 552 +11.893825604015609 553 +6.103292651887602E7 554 +30.861513970540496 555 +2.76825525814765E7 556 +2.127349968345691E7 557 +9719023.214790335 558 +4.362572664922104 559 +2.1273498892567933E7 560 +45.55295122062763 561 +19.214943439339212 562 +5.114352142717771E7 563 +2.7682552539640553E7 564 +5.7599633326025866E7 565 +1.2202734102108732E7 566 +5.759962754777615E7 567 +9719023.10978348 568 +9719023.131478565 569 +5.114352150051953E7 570 +5.114352313137377E7 571 +5.114352037914168E7 572 +1.2202733276986198E7 573 +3.7090213776948765E7 574 +4.59735498803913E7 575 +3.709021215068601E7 576 +5.759962771491455E7 577 +2.7682553294862375E7 578 +1.2202732943153184E7 579 +2.1273498989698533E7 580 +6.1032927225378506E7 581 +2.7682555606675062E7 582 +5.196250745708408 583 +6.103292838600789E7 584 +31.110426140877724 585 +5.759962743872689E7 586 +5.7599628423423275E7 587 +2.768255774403091E7 588 +2.768255237699016E7 589 +2.7682557831507005E7 590 +5.1143521259261794E7 591 +9719023.285177438 592 +2.768255749966389E7 593 +4.597355032776513E7 594 +6.103292676818215E7 595 +6.103292765901958E7 596 +2.7682551827339806E7 597 +2.127349831833844E7 598 +2.1273498369417276E7 599 +3.709021587054094E7 600 +6.103292750858647E7 601 +2.7682551832848746E7 602 +2.768255363083426E7 603 +9719023.22292005 604 +5.114352090950526E7 605 +5.759963039303699E7 606 +5.1143521741684794E7 607 +6.1032931219967194E7 608 +2.1273499021061104E7 609 +6.1032926695495225E7 610 +6.1032931893331155E7 611 +6.10329271386465E7 612 +2.1273499512879327E7 613 +2.768255162995979E7 614 +6.103292712958561E7 615 +9719023.329411559 616 +2.7682556104540266E7 617 +4.5973575128970616E7 618 +27.330725702730064 619 +6.103292674531662E7 620 +1.2202736802149417E7 621 +9719023.619746823 622 +5.1143520754668355E7 623 +3.709021260471606E7 624 +6.1032927170048274E7 625 +5.1143520640303016E7 626 +5.759962715843357E7 627 +4.053271106382381 628 +1.2202734116682822E7 629 +4.597354942386536E7 630 +2.127350099527855E7 631 +2.7682551737351898E7 632 +2.768255276580541E7 633 +2.1273499652460318E7 634 +6.103292719307782E7 635 +5.75996271098396E7 636 +75.04650533740856 637 +5.759964278507208E7 638 +5.759962798147487E7 639 +8.40042219107694 640 +3.709021291045477E7 641 +21.952765930224118 642 +3.7090213954914E7 643 +6.103292685780731E7 644 +3.70902443372848E7 645 +3.709021298646986E7 646 +3.709021229267092E7 647 +2.76825524942855E7 648 +6.103292921871322E7 649 +3.709021479760577E7 650 +6.103292831880903E7 651 +5.114352382197948E7 652 +1.2202733834016342E7 653 +3.709021281085691E7 654 +2.768255157897565E7 655 +6.103292694558859E7 656 +4.597355072353902E7 657 +5.114352101107491E7 658 +5.7599627127825625E7 659 +5.1143520751473956E7 660 +6.103292739661556E7 661 +4.597354978463502E7 662 +9.365735343069051 663 +1.2202733862144316E7 664 +2.1273498503113896E7 665 +2.1273498509460386E7 666 +1.2202733532281434E7 667 +8.628268510091942 668 +5.114352049925694E7 669 +3.709021257384388E7 670 +5.114352093253706E7 671 +5.75996278317761E7 672 +3.709021228465717E7 673 +9719023.975739943 674 +1.2202732967307737E7 675 +1.2202734126093725E7 676 +4.597354931898508E7 677 +2.1273500677747715E7 678 +3.709021477451674E7 679 +5.759962726412227E7 680 +56.718678795399896 681 +2.127349904207185E7 682 +4.597355083835686E7 683 +5.1143521211750045E7 684 +7.952785796385021 685 +4.597354924282112E7 686 +5.7599628005347095E7 687 +2.1273499016775433E7 688 +2.127349898329789E7 689 +4.597354898401705E7 690 +5.7599627208423436E7 691 +9719023.138554221 692 +1.2202735290369892E7 693 +3.709021267368493E7 694 +5.759962735355998E7 695 +9.383135377647458 696 +3.7829197829810433 697 +33.85766078459273 698 +1.2202734121993413E7 699 +4.5973548847815424E7 700 +5.1143521346790835E7 701 +6.1032928334728755E7 702 +4.597354915784334E7 703 +5.759962709625157E7 704 +1.2202733322663272E7 705 +5.1143521453109995E7 706 +5.114352037725015E7 707 +5.759963182469065E7 708 +2.1273503361067023E7 709 +1.220273465573737E7 710 +13.153447904494007 711 +2.7682559922457967E7 712 +1.2202736636921732E7 713 +1.2202734139300652E7 714 +3.709021275572794E7 715 +9719023.803198105 716 +2.7682554395703826E7 717 +2.1273498583104827E7 718 +1.2202732974353971E7 719 +3.709021762231091E7 720 +5.114352369190407E7 721 +2.1273498956819598E7 722 +20.054063962087273 723 +5.114352115170422E7 724 +2.7682551625419006E7 725 +2.768255226946022E7 726 +4.597356917817423E7 727 +5.1143520668541476E7 728 +5.7599629632290244E7 729 +4.5973549040649004E7 730 +22.887270437846187 731 +5.759962735612923E7 732 +2.768255325001292E7 733 +5.1143521019655034E7 734 +9719023.077492405 735 +2.7682552004217032E7 736 +9719023.247538274 737 +2.768255237442046E7 738 +3.709022083414804E7 739 +5.114352113178773E7 740 +9719023.071827576 741 +6.1032926689802684E7 742 +95.35519669509753 743 +1.2202733118103618E7 744 +2.1273499236757517E7 745 +3.709021216135179E7 746 +5.198888302460027 747 +2.7682551556092527E7 748 +4.597354880729009E7 749 +2.7682561304114614E7 750 +1.2202733629043281E7 751 +6.103292729018767E7 752 +4.5973549294092916E7 753 +9719023.083343579 754 +5.114352164364351E7 755 +4.59735493960844E7 756 +2.1273498558880642E7 757 +4.724064157605944 758 +5.1143521855114296E7 759 +5.759963002700283E7 760 +2.768255155167677E7 761 +3.709021530409186E7 762 +3.709021220316951E7 763 +3.709021278326439E7 764 +5.759962762202282E7 765 +2.127349924358547E7 766 +6.103292705340291E7 767 +4.597354963001001E7 768 +6.103292681543394E7 769 +9719023.00440087 770 +2.1273500148566123E7 771 +1.2202733203096839E7 772 +5.807604988818529 773 +1.2202733236007469E7 774 +26.16454788127745 775 +5.114352154845236E7 776 +4.866131171886266 777 +67.83343837974607 778 +1.2202734411245728E7 779 +2.1273498628735676E7 780 +5.1143521999419026E7 781 +3.709021509147911E7 782 +3.709021247855542E7 783 +9719023.01037977 784 +9719023.38110817 785 +22.013964228344406 786 +156.11471237934768 787 +2.1273498534806065E7 788 +2.768255248049105E7 789 +6.1032932124704994E7 790 +5.114352129251639E7 791 +6.103295683282918E7 792 +6.103292962018917E7 793 +2.7682553033935506E7 794 +1.220273509236596E7 795 +5.759962754037768E7 796 +18.012118618462754 797 +4.5973549144636974E7 798 +4.652272695580429 799 +4.597354911947667E7 800 +4.597354915009873E7 801 +5.759964018590911E7 802 +9719023.083809765 803 +9719023.388570976 804 +6.103292677153257E7 805 +2.768255539665165E7 806 +3.709021211615367E7 807 +3.709021215698861E7 808 +4.5973549671403036E7 809 +4.5973549112987615E7 810 +3.7090213227810554E7 811 +9719023.103017276 812 +3.709021639635705E7 813 +9719023.23256335 814 +5.11435207287492E7 815 +2.7682551986369785E7 816 +5.114352093769953E7 817 +3.709023323536767E7 818 +1.2202737617674781E7 819 +2.7682552402665023E7 820 +4.597354901616503E7 821 +9719023.463186353 822 +3.709021429751574E7 823 +2.127349853126915E7 824 +4.597354913532525E7 825 +2.1273504862804815E7 826 +4.4071945832091055 827 +3.7090212752095655E7 828 +1.2202733336662488E7 829 +4.59735570849834E7 830 +2.127349854837954E7 831 +5.114352042115381E7 832 +2.7682551735205594E7 833 +3.709021243799115E7 834 +2.1273499144541893E7 835 +6.10329287991833E7 836 +9719023.336994197 837 +2.76825524057453E7 838 +2.1273498772914063E7 839 +2.1273498824623983E7 840 +6.103292788685157E7 841 +9719023.108682128 842 +2.768255293668597E7 843 +1.2202733934191635E7 844 +6.103292689207704E7 845 +5.114352115223807E7 846 +2.7682551545384165E7 847 +2.7682552912171066E7 848 +5.114352099492527E7 849 +3.7090212515707746E7 850 +6.103292710675166E7 851 +9719023.728757141 852 +9719023.03331562 853 +12.652144644155406 854 +4.597354909318116E7 855 +9719023.349298075 856 +5.759964326500281E7 857 +1.2202732956097476E7 858 +2.768255349588581E7 859 +9719025.309208427 860 +11.200963577783986 861 +2.127349858922743E7 862 +6.1032926520041294E7 863 +42.52628392301198 864 +9719023.119417535 865 +4.597354919668739E7 866 +1.2202735816168185E7 867 +2.7682552457347345E7 868 +5.114352132069005E7 869 +9719023.132180464 870 +2.7682562998823214E7 871 +17.061112732579478 872 +6.103292715513088E7 873 +5.7599627698200904E7 874 +9719023.103232576 875 +9719023.20197469 876 +5.1143520975275874E7 877 +5.114352114080233E7 878 +5.114352049624623E7 879 +18.36414648491376 880 +6.103293082194192E7 881 +3.709022020050289E7 882 +4.597354927232259E7 883 +9719023.218828095 884 +4.5973549146774165E7 885 +26.719831707041436 886 +5.7599627451505445E7 887 +5.114352355119651E7 888 +5.114352105528401E7 889 +9.808507373968759 890 +4.1863979940654685 891 +6.1032929248015895E7 892 +2.1273498316863257E7 893 +2.7682556658799555E7 894 +4.5973549267170906E7 895 +2.127349879915766E7 896 +2.127349857452364E7 897 +9719023.2473468 898 +2.768255255009089E7 899 +1.2202734333755784E7 900 +4.5973549110542E7 901 +2.7682552757493217E7 902 +9719023.201120203 903 +3.709021274632047E7 904 +6.1032926837458774E7 905 +14.446426712239925 906 +2.1273501070167527E7 907 +6.103292676736421E7 908 +2.1273499781454016E7 909 +5.75996274800239E7 910 +5.759962719987244E7 911 +1.2202734266067559E7 912 +9719023.088691015 913 +73.51361985789526 914 +3.709021331289028E7 915 +5.114352135484605E7 916 +1.2202733281890688E7 917 +6.103292874053738E7 918 +5.114352085454468E7 919 +5.114352118123197E7 920 +5.759962746121619E7 921 +9719023.529044293 922 +2.768255202331097E7 923 +2.1273498949369196E7 924 +3.709021321169044E7 925 +1.2202733379513659E7 926 +2.768255347321771E7 927 +1.2202733050741157E7 928 +4.5973549234005295E7 929 +6.103292902741389E7 930 +1.220274550400491E7 931 +6.103292659816241E7 932 +3.709021298302409E7 933 +5.7599627342688605E7 934 +9719023.034559835 935 +1.2202733707193037E7 936 +5.114352134713739E7 937 +3.2318305153342726 938 +6.103292755206176E7 939 +1.2202733160070673E7 940 +5.114352197602183E7 941 +4.597354908103986E7 942 +5.7599627405082665E7 943 +5.114352116558762E7 944 +6.276895649106817 945 +71.54385012449592 946 +5.114352091277031E7 947 +5.114352038489418E7 948 +5.114352173358392E7 949 +3.709021536661905E7 950 +6.103292698015656E7 951 +6.103292726916914E7 952 +9719023.383784555 953 +3.709021211555077E7 954 +6.103292740123264E7 955 +5.759962819699827E7 956 +11.374087146171282 957 +4.597354888838407E7 958 +2.1273499353416167E7 959 +22.20124153267974 960 +9719023.45947622 961 +2.7682553813802604E7 962 +4.59735489870904E7 963 +9719023.0850069 964 +2.127349944309255E7 965 +5.114352054955019E7 966 +1.2202733514831178E7 967 +2.768255406056055E7 968 +17.07573556340141 969 +2.7682552560899466E7 970 +2.768255180213553E7 971 +1.2202748635949383E7 972 +4.597354885047151E7 973 +1.2202734280713262E7 974 +5.114352142299551E7 975 +2.76825555113875E7 976 +1.2202733194345728E7 977 +19.73434957733601 978 +9719023.017604645 979 +5.759962735603023E7 980 +3.7090212612219945E7 981 +3.709021243335994E7 982 +5.759962753282497E7 983 +6.103292985006058E7 984 +5.114352332682359E7 985 +5.7599627513674766E7 986 +1.2202738188728498E7 987 +3.709021384713939E7 988 +13.058851152523658 989 +2.768255196905446E7 990 +3.709021233183203E7 991 +4.5973548988380276E7 992 +3.709021206928361E7 993 +6.103292819763043E7 994 +1.2202733689097568E7 995 +2.768255170442511E7 996 +1.220273462743898E7 997 +6.103292968983686E7 998 +6.103292911600844E7 999 +5.75996284978274E7 1000 +9719023.02916965 1001 +5.759962741766794E7 1002 +2.1273498949253187E7 1003 +3.7090212796604425E7 1004 +2.1273498994316768E7 1005 +2.1273509211978417E7 1006 +2.768255208938465E7 1007 +4.597354879609129E7 1008 +5.1143520557714365E7 1009 +3.70902125299074E7 1010 +6.103292726757432E7 1011 +2.1273499043350853E7 1012 +3.709021639794294E7 1013 +2.127349840825887E7 1014 +2.768255336122664E7 1015 +1.2202733327163635E7 1016 +8.191958082866458 1017 +6.103292707281378E7 1018 +2.1273498501325715E7 1019 +10.36615837154663 1020 +4.5973557027819656E7 1021 +5.1143520474917255E7 1022 +12.828143682493558 1023 +4.597354897315659E7 1024 +2.768255162541424E7 1025 +6.103292669575132E7 1026 +1.2202733539476875E7 1027 +2.1273498658975568E7 1028 +5.759962771090487E7 1029 +5.1143926749151595E7 1030 +6.51973688752495 1031 +1.2202735358325208E7 1032 +5.1143520600998834E7 1033 +2.768255290011677E7 1034 +5.979993620666563 1035 +1.2202733260407114E7 1036 +1.2202734460261285E7 1037 +2.7682598353291374E7 1038 +6.103292736903721E7 1039 +2.127349946252993E7 1040 +6.103292677733594E7 1041 +3.70902124286869E7 1042 +6.103293559736628E7 1043 +2.127349905169466E7 1044 +6.1032927226237856E7 1045 +2.1273498931799825E7 1046 +2.7682552049352113E7 1047 +2.1273498564689595E7 1048 +3.709021238593491E7 1049 +10.460006286148888 1050 +3.709021270795389E7 1051 +1.220273338427937E7 1052 +2.7682552428612933E7 1053 +5.114352106535622E7 1054 +9719023.703551611 1055 +4.597354988264425E7 1056 +6.1032927862165876E7 1057 +2.1273499267451823E7 1058 +3.709021390751708E7 1059 +6.103292662166977E7 1060 +9719023.237974437 1061 +5.114352110240961E7 1062 +2.7682551651147515E7 1063 +4.597354915233902E7 1064 +4.597354992798874E7 1065 +2.1273498440635387E7 1066 +2.12734990078526E7 1067 +1.2202751464737643E7 1068 +3.70902199364297E7 1069 +2.127349907743969E7 1070 +3.709021226234426E7 1071 +9719023.644016104 1072 +2.768255251156714E7 1073 +5.114352089858067E7 1074 +1.2202735220397277E7 1075 +5.7599627054612234E7 1076 +9719023.084311115 1077 +4.045252638352985 1078 +5.114352042517583E7 1079 +9719023.04347347 1080 +1.2202734731546862E7 1081 +5.759962761143143E7 1082 +3.709021330898962E7 1083 +5.114352090968247E7 1084 +6.103292741061275E7 1085 +2.127349891779104E7 1086 +2.768255267434381E7 1087 +4.381576621323248 1088 +2.768255154222645E7 1089 +3.7090214174970046E7 1090 +6.103292763246406E7 1091 +9719023.539819138 1092 +2.7682551597193416E7 1093 +2.1273500626368057E7 1094 +9719023.034239976 1095 +9719023.324780164 1096 +1.2202734392302217E7 1097 +4.597354961506841E7 1098 +3.7090214107182726E7 1099 +5.1143521598300084E7 1100 +9719024.162386598 1101 +3.709021368432664E7 1102 +9719023.306901528 1103 +5.759962728120907E7 1104 +6.1032926858796686E7 1105 +5.759962708881962E7 1106 +5.759962965760308E7 1107 +5.75996271775722E7 1108 +4.5973549363337144E7 1109 +5.1143520555233054E7 1110 +6.103292744584893E7 1111 +4.597354881227479E7 1112 +6.1032926888867564E7 1113 +9719023.321694724 1114 +5.11435210976709E7 1115 +4.5328730092000225 1116 +5.114352058166353E7 1117 +5.114352102003676E7 1118 +5.114352081043387E7 1119 +1.2202733998989841E7 1120 +2.7682551777114328E7 1121 +2.768255214439479E7 1122 +5.759962713008532E7 1123 +5.114352148455817E7 1124 +5.759962840485486E7 1125 +6.1032927963146016E7 1126 +2.768255268534055E7 1127 +4.597354895631602E7 1128 +6.1032926557255775E7 1129 +3.7090215096983835E7 1130 +6.103292716214134E7 1131 +2.127349928261824E7 1132 +5.759962741209825E7 1133 +8.467282740895683 1134 +4.597354884816963E7 1135 +3.70902133916489E7 1136 +2.7682551982121732E7 1137 +5.759962725905757E7 1138 +9719023.252653232 1139 +2.768255249358863E7 1140 +4.597354950178192E7 1141 +2.7682552782693025E7 1142 +1.2202733636093143E7 1143 +1.220273349056091E7 1144 +4.5973549323585704E7 1145 +6.10329446036018E7 1146 +6.103293906687433E7 1147 +3.7090212848538585E7 1148 +6.68930235845048 1149 +4.59735492793816E7 1150 +9719023.345916528 1151 +3.709021219497791E7 1152 +1.2202734191564297E7 1153 +5.1143520811496384E7 1154 +3.7090212134583175E7 1155 +4.5973550332720175E7 1156 +1.2202737172630934E7 1157 +4.597354905361417E7 1158 +9719061.745733995 1159 +4.597355002008778E7 1160 +7.084864853987911 1161 +6.103293034736322E7 1162 +5.114352185541995E7 1163 +2.1273498904614713E7 1164 +6.103292764806503E7 1165 +3.7090213231165916E7 1166 +6.103292668982674E7 1167 +1.22027336345535E7 1168 +3.709021509379559E7 1169 +6.103292654341103E7 1170 +4.59735512666312E7 1171 +5.7599627263349734E7 1172 +6.1193502446468555 1173 +4.5973548953036405E7 1174 +1.2202739810842237E7 1175 +9719023.140424343 1176 +9719023.216691466 1177 +1.220273408293704E7 1178 +4.5973549858165E7 1179 +4.597354926406099E7 1180 +3.709021242574747E7 1181 +2.1273498760025337E7 1182 +1.2202733787915248E7 1183 +2.7682551984159477E7 1184 +2.1273498753705963E7 1185 +5.114352150491203E7 1186 +3.709021209725812E7 1187 +1.2202733687551944E7 1188 +4.597354929086116E7 1189 +5.11435220865818E7 1190 +2.7682559187894106E7 1191 +9719023.241787598 1192 +5.1143521435045E7 1193 +9719025.079981372 1194 +1.2202734054590534E7 1195 +3.7090212242726184E7 1196 +1.2202733286328241E7 1197 +1.2202733636060076E7 1198 +3.709021330847761E7 1199 +5.759962746603724E7 1200 +3.709021468615917E7 1201 +3.7090212693712175E7 1202 +9719031.021350116 1203 +16.07120507287506 1204 +9719023.106619723 1205 +2.1273501372467104E7 1206 +2.127349861275373E7 1207 +3.709021286318887E7 1208 +2.7682559426037487E7 1209 +1.220273291569977E7 1210 +13.190869978340421 1211 +4.5973548980303556E7 1212 +5.759962720706447E7 1213 +6.629970178683336 1214 +5.114352120789788E7 1215 +5.114352150015575E7 1216 +5.75996272997855E7 1217 +6.1032926694041505E7 1218 +5.114352124152029E7 1219 +4.5973548836562075E7 1220 +1.220273518646984E7 1221 +6.103292689070521E7 1222 +1.2202736698385675E7 1223 +6.103292663790523E7 1224 +4.613033807383756 1225 +4.49261609490798 1226 +9719023.047228832 1227 +2.127349917934935E7 1228 +2.7682551742505554E7 1229 +5.759962715105785E7 1230 +2.127349947656577E7 1231 +2.1273498479874913E7 1232 +4.597354918639117E7 1233 +186.97056969930966 1234 +9.327328242912072 1235 +5.759962985360935E7 1236 +3.709021497936077E7 1237 +1.2202734333929423E7 1238 +4.59735498004539E7 1239 +2.1273498647599757E7 1240 +6.103293149466865E7 1241 +6.103292890866772E7 1242 +6.103292789789715E7 1243 +5.1143528829722434E7 1244 +2.127351214760474E7 1245 +1.220273540339486E7 1246 +4.597354913015633E7 1247 +4.597355064673871E7 1248 +4.59735498554588E7 1249 +2.1273498369783793E7 1250 +6.103292753480344E7 1251 +9719023.838585967 1252 +3.7090212143444486E7 1253 +9719023.425405923 1254 +5.1143521339069225E7 1255 +6.1032926845151536E7 1256 +5.759962715656641E7 1257 +9719023.828482598 1258 +5.759962806111902E7 1259 +6.1032926781365976E7 1260 +2.12734984742036E7 1261 +4.59735606166449E7 1262 +2.1273498377274048E7 1263 +4.597354972784421E7 1264 +4.597354944336552E7 1265 +9719023.163407905 1266 +5.114352118816993E7 1267 +4.597354890907681E7 1268 +6.10329310562451E7 1269 +5.7599629192639485E7 1270 +4.3475630648628965 1271 +9719023.257260013 1272 +2.1273499041624926E7 1273 +9719023.277681038 1274 +5.1143520799017705E7 1275 +9719024.818884613 1276 +4.597354900122596E7 1277 +3.709021399237908E7 1278 +9719023.01696822 1279 +5.11435209170314E7 1280 +3.709021327322205E7 1281 +2.1273498992046382E7 1282 +3.709021259282063E7 1283 +4.750335059415751 1284 +9719023.268967781 1285 +4.597354933987148E7 1286 +3.177967259445166 1287 +5.114352057541168E7 1288 +40.75962813642748 1289 +19.67137697360268 1290 +4.334746131701832 1291 +2.7682551676603816E7 1292 +2.127349992410676E7 1293 +5.759962727031881E7 1294 +3.7090212544708125E7 1295 +5.7599627244555704E7 1296 +9719023.431553464 1297 +5.114352080881386E7 1298 +2.7682677421654485E7 1299 +4.59735494048003E7 1300 +9719023.613835547 1301 +2.7682551768964574E7 1302 +5.7599627642464474E7 1303 +4.5973549561514206E7 1304 +6.103292694796191E7 1305 +5.759962760782165E7 1306 +4.597354976582804E7 1307 +5.759962713376847E7 1308 +28.105717461284474 1309 +9719023.359789101 1310 +9.281871827803363 1311 +4.597354893262083E7 1312 +9719023.333911877 1313 +5.759962732703678E7 1314 +3.709021576460447E7 1315 +4.59735494266404E7 1316 +1.2202735626115292E7 1317 +3.6771294124016842 1318 +6.103292662839561E7 1319 +3.709022556215019E7 1320 +4.5973548883756E7 1321 +9719023.585090252 1322 +6.103292662127083E7 1323 +5.759962743706434E7 1324 +2.1273498527086936E7 1325 +2.768255231878554E7 1326 +2.1273498856750254E7 1327 +3.7090215298697345E7 1328 +9719023.035367526 1329 +4.5973549529022515E7 1330 +6.058535977945166 1331 +5.114352076582994E7 1332 +9719023.180388082 1333 +5.114352349968923E7 1334 +5.759962764715658E7 1335 +2.7682559201531384E7 1336 +2.76825527140653E7 1337 +6.1032928811721414E7 1338 +5.114352075421827E7 1339 +3.709021415303368E7 1340 +6.103292666439531E7 1341 +4.59735566902162E7 1342 +6.103292669227215E7 1343 +9719023.219405577 1344 +4.597354943706218E7 1345 +4.597354893679873E7 1346 +3.70902130122996E7 1347 +2.7682551621645227E7 1348 +2.768255374628221E7 1349 +9719023.340369202 1350 +5.114352166871062E7 1351 +4.5973549411710404E7 1352 +5.759962715127789E7 1353 +1.2202733038715336E7 1354 +2.7682552087486226E7 1355 +4.59735493046866E7 1356 +81.4252132151079 1357 +2.7682553182032835E7 1358 +9719023.140696503 1359 +6.103292766000105E7 1360 +3.70902121567312E7 1361 +3.70902172822484E7 1362 +4.597355141011643E7 1363 +1.220273314225173E7 1364 +5.759962755262022E7 1365 +2.7682554850901235E7 1366 +9719023.004010297 1367 +4.5973549776195355E7 1368 +2.7682552078491934E7 1369 +2.1273499022534486E7 1370 +5.114352049915681E7 1371 +3.709021318622593E7 1372 +5.759962879904495E7 1373 +5.114352113393251E7 1374 +9719023.028458925 1375 +1.2202735043538287E7 1376 +6.103293995443978E7 1377 +2.7682596772403687E7 1378 +4.597354908221207E7 1379 +9719023.11503766 1380 +2.127349854742733E7 1381 +8.08385192791373 1382 +2.7683541882547326E7 1383 +9719023.018490879 1384 +1.2202747521676766E7 1385 +2.127349883326325E7 1386 +9719024.590665458 1387 +2.1273499238196976E7 1388 +38.237335369149164 1389 +3.709022646391417E7 1390 +3.025762902322075 1391 +5.114352048195963E7 1392 +1.220273525157704E7 1393 +25.03730002298632 1394 +9719023.177221958 1395 +9719023.300565224 1396 +21.179495768740164 1397 +2.1273499035475407E7 1398 +4.597355005434541E7 1399 +6.10329271789926E7 1400 +2.7682555120997775E7 1401 +5.7599628859392E7 1402 +6.1032929897975825E7 1403 +3.709021349680636E7 1404 +1.2202734308880562E7 1405 +2.768255262199475E7 1406 +5.759966689166033E7 1407 +2.7682552869614203E7 1408 +3.7090212741562046E7 1409 +2.1273502682666756E7 1410 +5.114360364017246E7 1411 +5.759962722196293E7 1412 +5.114352109730429E7 1413 +2.768256555560389E7 1414 +5.759962717785106E7 1415 +4.5973549435154445E7 1416 +5.7599627355847634E7 1417 +9719023.24233171 1418 +3.7090213244665675E7 1419 +2.768255311785785E7 1420 +5.114352054848644E7 1421 +2.1273499070760254E7 1422 +4.5973551176412195E7 1423 +6.103304671712226E7 1424 +6.1032926777757734E7 1425 +5.114352059431913E7 1426 +8.382269674309196 1427 +5.1143520475172214E7 1428 +9719023.47621954 1429 +4.597354891049016E7 1430 +5.759962708221764E7 1431 +2.768255202825824E7 1432 +2.768255160860927E7 1433 +3.709021268087202E7 1434 +5.114352176672571E7 1435 +132.1403930405947 1436 +2.1273499643403392E7 1437 +2.7682551730591994E7 1438 +5.114352038954381E7 1439 +9719023.795001445 1440 +5.114352493140832E7 1441 +6.103292681835701E7 1442 +5.114352279236895E7 1443 +3.7090213812753804E7 1444 +3.709021660165794E7 1445 +6.103292718595637E7 1446 +1.220273331321817E7 1447 +9719023.85131497 1448 +2.1273499541788936E7 1449 +5.7599627341935754E7 1450 +9719023.172909895 1451 +2.1273498646363378E7 1452 +4.597354923105579E7 1453 +6.103292668585996E7 1454 +5.759962788518186E7 1455 +5.1143524548042E7 1456 +3.709021260487474E7 1457 +9719023.408801025 1458 +9719023.422883188 1459 +4.597354945944878E7 1460 +1.2202735626208318E7 1461 +2.7682552772007763E7 1462 +3.709021373555152E7 1463 +5.114352088496616E7 1464 +5.114352038118853E7 1465 +4.597354898569399E7 1466 +2.127349977771392E7 1467 +9719023.249250215 1468 +1.220273346014933E7 1469 +3.709021208148799E7 1470 +4.597354958007703E7 1471 +5.759962721495922E7 1472 +2.7682551973585688E7 1473 +131.26780607791602 1474 +3.709021252110551E7 1475 +2.127349836044533E7 1476 +1.220273289933651E7 1477 +4.597354903666327E7 1478 +5.759962706275236E7 1479 +5.114352091491646E7 1480 +1.220274054654132E7 1481 +5.759962724936969E7 1482 +4.597354888254032E7 1483 +5.759962731785021E7 1484 +5.759962849258633E7 1485 +12.91073703730201 1486 +5.759962715043503E7 1487 +6.103292667578497E7 1488 +9719023.443111518 1489 +6.103292666228597E7 1490 +4.597354893095512E7 1491 +2.1273498694414947E7 1492 +4.5973549335313395E7 1493 +2.127349881479965E7 1494 +5.114352242161912E7 1495 +22.694298129790297 1496 +2.768255280623375E7 1497 +2.1273498974121023E7 1498 +4.597354916724571E7 1499 +14.098401178263813 1500 +5.114352125636672E7 1501 +2.127349841304053E7 1502 +5.1143520887964115E7 1503 +9719023.214127135 1504 +4.5973549329367846E7 1505 +9719023.177689768 1506 +3.70902140668476E7 1507 +2.12734988801832E7 1508 +6.103292961477285E7 1509 +5.7599627521859854E7 1510 +2.1273498302456614E7 1511 +2.768255165871241E7 1512 +1.220273315866579E7 1513 +5.759962705265985E7 1514 +1.2202736493471872E7 1515 +7.472538634218784 1516 +3.709021280344476E7 1517 +2.768255547068634E7 1518 +9719023.217197608 1519 +3.70902149534668E7 1520 +4.5973550355978794E7 1521 +6.103293484008162E7 1522 +2.768255176474312E7 1523 +9719023.77708084 1524 +2.1273498837766998E7 1525 +1.2202739732507195E7 1526 +4.5973549332922675E7 1527 +2.127349936059282E7 1528 +6.1032927383190975E7 1529 +2.1273498553834E7 1530 +4.5973550037559055E7 1531 +2.7682551560458776E7 1532 +6.10329276912913E7 1533 +1.2202732872577965E7 1534 +5.1143520540847E7 1535 +1.2202734337059403E7 1536 +5.11435207865266E7 1537 +2.1273501861169826E7 1538 +2.768255424532107E7 1539 +5.114352126326248E7 1540 +3.709021221568441E7 1541 +6.103292795706114E7 1542 +5.114352105748738E7 1543 +3.7090212439171545E7 1544 +5.759962704081661E7 1545 +9719024.396471515 1546 +9719023.247475235 1547 +5.114352068252835E7 1548 +5.759962735736477E7 1549 +5.759962707227654E7 1550 +3.709021654838854E7 1551 +2.768255208769308E7 1552 +1.2202733602103384E7 1553 +6.348192996497525 1554 +9719023.467872137 1555 +4.5973549811614804E7 1556 +2.768255255498111E7 1557 +5.114352114869866E7 1558 +2.768255481351747E7 1559 +3.709021215549251E7 1560 +5.1143520423223905E7 1561 +3.709021341278381E7 1562 +3.709021219409154E7 1563 +1.2202733286548864E7 1564 +5.759963205817086E7 1565 +2.127350929004719E7 1566 +2.7682557931771535E7 1567 +1.220273310258037E7 1568 +6.103292654120683E7 1569 +1.22027470020019E7 1570 +3.7090212538894124E7 1571 +6.103292669638012E7 1572 +9719023.297747362 1573 +2.1273501392472003E7 1574 +2.127349880082996E7 1575 +9719025.901443638 1576 +9719024.963916417 1577 +2.1273498557937134E7 1578 +3.709021244753071E7 1579 +3.7090212677890085E7 1580 +3.709021250161383E7 1581 +2.1273498800684653E7 1582 +2.7682551723674532E7 1583 +5.11435209982273E7 1584 +3.709021210201673E7 1585 +2.1273498779844675E7 1586 +171.51164237169527 1587 +2.127349886498757E7 1588 +9719023.015825255 1589 +2.7682567106581982E7 1590 +1.2202734165124586E7 1591 +6.10331625636708E7 1592 +1.220273516210554E7 1593 +4.597354905585129E7 1594 +9719023.018651552 1595 +1.220273367185915E7 1596 +5.11435209973016E7 1597 +1.2202733866503246E7 1598 +4.5973549541641645E7 1599 +3.7090212322106004E7 1600 +2.127350002314403E7 1601 +3.7090212172613785E7 1602 +9719023.483939601 1603 +5.759962720748382E7 1604 +4.5973549949568346E7 1605 +9719023.27953529 1606 +9719023.719969254 1607 +5.114352132590097E7 1608 +2.7682568030133985E7 1609 +2.1273499024170183E7 1610 +1330.8570536144698 1611 +4.597354938327906E7 1612 +6.103294407158887E7 1613 +7.818836384587803 1614 +6.103292735406427E7 1615 +1.2202732944360072E7 1616 +5.759962751341261E7 1617 +1.220273344994707E7 1618 +1.2202733380765434E7 1619 +5.7599627928065695E7 1620 +4.597354959636015E7 1621 +9719023.33562444 1622 +85.89137697984864 1623 +4.047225976746057 1624 +2.7682555071417943E7 1625 +6.103292705701756E7 1626 +3.709021554058565E7 1627 +5.114352164118683E7 1628 +3.709021554149286E7 1629 +4.669419368251814 1630 +5.1143520814330034E7 1631 +2.127350079658883E7 1632 +5.114352059973395E7 1633 +2.768255193485164E7 1634 +3.7090213107779495E7 1635 +3.709021397403089E7 1636 +3.709021342870269E7 1637 +6.1032927233427145E7 1638 +2.768255679913089E7 1639 +4.597355093188427E7 1640 +1.2202737126250105E7 1641 +14.185700942783365 1642 +5.114352045013263E7 1643 +4.597354924778939E7 1644 +3.709021283329913E7 1645 +1.2202733925000545E7 1646 +5.114352124785717E7 1647 +6.103293721260983E7 1648 +44.52283607566138 1649 +6.103292680195351E7 1650 +1.2202732957191968E7 1651 +2.7682554115897927E7 1652 +44.4636184914688 1653 +2.1273498927279223E7 1654 +37.06548898276591 1655 +2.768255215173905E7 1656 +3.709021205555707E7 1657 +1.2202733228269491E7 1658 +9719023.701672727 1659 +3.0258419955078857 1660 +5.7599627373089306E7 1661 +1.220273325074297E7 1662 +5.759962731622758E7 1663 +6.103292689446248E7 1664 +4.597354923573073E7 1665 +6.103292851292787E7 1666 +6.103292663769665E7 1667 +5.759962721634196E7 1668 +5.759962736604521E7 1669 +5.759962706606463E7 1670 +6.103292782583743E7 1671 +9719023.247267568 1672 +3.709021221861252E7 1673 +4.5973549005524784E7 1674 +5.75996272505383E7 1675 +2.1273498710246082E7 1676 +1.2202733147682128E7 1677 +5.114352116126231E7 1678 +15.363469871137456 1679 +6.103292710289005E7 1680 +3.7090218121916756E7 1681 +5.759962722884519E7 1682 +6.1032930955447465E7 1683 +3.709021351971756E7 1684 +9719024.072864877 1685 +2.1273499058608513E7 1686 +2.127350208462472E7 1687 +6.103292726041402E7 1688 +1.2202734316982644E7 1689 +5.114352070282754E7 1690 +3.7090213089810714E7 1691 +4.597354915100318E7 1692 +9719023.344978055 1693 +2.768255319816723E7 1694 +17.884386205435984 1695 +5.759962734234108E7 1696 +1.2202734139935289E7 1697 +4.597354950768187E7 1698 +2.76825516954009E7 1699 +2.127349857575564E7 1700 +1.2202734026671046E7 1701 +3.709021241860584E7 1702 +3.709021265217385E7 1703 +5.114352073922699E7 1704 +2.1273499943997655E7 1705 +4.597354948853048E7 1706 +4.597355362888609E7 1707 +2.7682552105942115E7 1708 +9719023.10155216 1709 +5.75996270996788E7 1710 +2.768255258811019E7 1711 +9719023.299163563 1712 +6.1032927600000665E7 1713 +5.75996294327091E7 1714 +3.709021314268716E7 1715 +1.22027355806123E7 1716 +7.148350586612579 1717 +1.220273564000998E7 1718 +3.709021306110182E7 1719 +2.1273498943735298E7 1720 +17.70292258687199 1721 +4.597354944259119E7 1722 +9719023.203864908 1723 +5.759962706137372E7 1724 +1.2202733616170336E7 1725 +177.12246153386843 1726 +9719031.03994348 1727 +9719023.486208467 1728 +2.7682552540825125E7 1729 +5.114352063363875E7 1730 +5.759962707393162E7 1731 +2.1273498706069235E7 1732 +1.220273939474343E7 1733 +19.789859202826193 1734 +9719023.253213093 1735 +3.709021243766573E7 1736 +9719023.15373547 1737 +3.690454380621082 1738 +6.1032926608321495E7 1739 +6.103292763152491E7 1740 +9719024.350002233 1741 +9719023.474600162 1742 +1.2202733062335731E7 1743 +5.1143524387906596E7 1744 +1.220273491737076E7 1745 +2.1273498958590765E7 1746 +53.92419792139237 1747 +4.597354930279796E7 1748 +5.114352048907925E7 1749 +2.1273499178239383E7 1750 +1.2202734140848614E7 1751 +2.1273498537547667E7 1752 +4.597354937315811E7 1753 +5.11435213961205E7 1754 +1.2202733505193628E7 1755 +11.989892114941572 1756 +2.768255178049458E7 1757 +9719023.03905795 1758 +2.7682553077405293E7 1759 +3.709021308174569E7 1760 +4.5973549041865855E7 1761 +4.5973549322207846E7 1762 +5.759962834227105E7 1763 +5.7599630922725506E7 1764 +2.7682552083502643E7 1765 +4.5973549658830285E7 1766 +5.1143520972041644E7 1767 +3.709021292213307E7 1768 +1.2202734885704337E7 1769 +4.597354895681495E7 1770 +2.768255172540358E7 1771 +9719023.024379116 1772 +5.1143521036937416E7 1773 +4.5973549387221806E7 1774 +5.114352513561698E7 1775 +6.103292766772302E7 1776 +5.7599627404182576E7 1777 +9719023.084331548 1778 +2.1273498957724083E7 1779 +9719023.307542427 1780 +2.7682551861248527E7 1781 +6.103292662918581E7 1782 +4.5973548879256316E7 1783 +2.7682557425715156E7 1784 +1.2202791542811466E7 1785 +2.127349887329115E7 1786 +2.7682552511770394E7 1787 +3.7090212727608405E7 1788 +3.709021308240699E7 1789 +2.7682554385297436E7 1790 +9.90810305910419 1791 +4.597354912552108E7 1792 +3.7090215514757946E7 1793 +5.114352037253642E7 1794 +2.1273498837005794E7 1795 +4.597354960608602E7 1796 +11.501478006993382 1797 +3.70902124073893E7 1798 +5.1143520998161845E7 1799 +3.857622938043598 1800 +10.467718385699598 1801 +9719023.122549241 1802 +5.759962713398421E7 1803 +3.709021260879713E7 1804 +3.70902121402521E7 1805 +2.1273499661217872E7 1806 +5.759963000507787E7 1807 +1.2202733694777131E7 1808 +5.156985441942215 1809 +2.127350493129706E7 1810 +5.1143520536463484E7 1811 +2.7682552414062887E7 1812 +6.103292770715401E7 1813 +9719023.040167866 1814 +2.768255212166289E7 1815 +4.479500148688061 1816 +5.114352045298942E7 1817 +9719023.357418722 1818 +2.127349829957675E7 1819 +5.114352080416396E7 1820 +6.1032926798065454E7 1821 +41.199660968149075 1822 +9719023.38896755 1823 +5.1143520742309116E7 1824 +6.103292800870012E7 1825 +9719023.083611116 1826 +1.220273421523775E7 1827 +4.597354897290353E7 1828 +1.2202733584269335E7 1829 +6.1032927429499984E7 1830 +5.61349790174328 1831 +19.33465356722359 1832 +2.7682551695407312E7 1833 +30.737671818248433 1834 +6.103292777802366E7 1835 +3.709021219815647E7 1836 +9719025.382025655 1837 +2.127350101293206E7 1838 +2.127349933047616E7 1839 +6.879823098931386 1840 +9719023.161546757 1841 +1.2202736369640911E7 1842 +1.2202733292847244E7 1843 +9719023.125593418 1844 +2.1273498330612674E7 1845 +5.7599655942715466E7 1846 +1.2202732995690426E7 1847 +13.205216624053598 1848 +1.2202743754881097E7 1849 +9719023.466488369 1850 +2.768255291653848E7 1851 +5.759962791806742E7 1852 +1.220273317521478E7 1853 +6.103292706658126E7 1854 +5.114352043770721E7 1855 +2.7682552537028246E7 1856 +2.768255166631606E7 1857 +3.709021316712138E7 1858 +2.127349850651816E7 1859 +3.709027857379043E7 1860 +2.7682553134335414E7 1861 +6.1032927270257056E7 1862 +9719023.042303145 1863 +2.127349868593558E7 1864 +6.103292792116039E7 1865 +3.6610045099092794 1866 +2.1273499227188714E7 1867 +1.2202733085247282E7 1868 +4.5973548831611216E7 1869 +9719023.11094495 1870 +2.1273498672251012E7 1871 +8.945470561821036 1872 +9719023.06171944 1873 +5.114352063340521E7 1874 +9719023.176165702 1875 +4.597354979583768E7 1876 +6.1032926959200166E7 1877 +6.10329290710095E7 1878 +5.1143521177327886E7 1879 +2.127350009537619E7 1880 +5.759962731797565E7 1881 +5.759962718526259E7 1882 +1.2202733119910775E7 1883 +3.709021221652499E7 1884 +27.325229126964583 1885 +6.103292846722374E7 1886 +1.2202733301233329E7 1887 +6.103292963834904E7 1888 +3.709021209150123E7 1889 +5.114352047987879E7 1890 +5.114352108770088E7 1891 +6.1032926769763954E7 1892 +5.114352122707798E7 1893 +5.759962701979101E7 1894 +4.597355245852812E7 1895 +10.215210619282123 1896 +5.158006000023696 1897 +5.114352102630055E7 1898 +3.709021203669539E7 1899 +11.07775891066429 1900 +5.1143520930346206E7 1901 +5.759962746382771E7 1902 +6.103292979429094E7 1903 +5.114352046370965E7 1904 +3.709021223839458E7 1905 +9719023.505857939 1906 +6.0190674296779365 1907 +2.768255581021463E7 1908 +6.103292999870596E7 1909 +3.709021354820682E7 1910 +9719023.568640849 1911 +9719023.250680113 1912 +4.739643374807136 1913 +1.2202733634371324E7 1914 +6.1032927021871045E7 1915 +7.400921233764215 1916 +6.103292761828033E7 1917 +2.7682585582598466E7 1918 +9719023.0550165 1919 +4.5973584508924425E7 1920 +216.99486453924385 1921 +2.127349834624324E7 1922 +6.103292733229742E7 1923 +18.771238179893103 1924 +6.103292701044616E7 1925 +4.597354881615864E7 1926 +1.220273368188244E7 1927 +5.7599636951360375E7 1928 +6.10329273242143E7 1929 +6.103292674692403E7 1930 +1.2202734023932379E7 1931 +3.7090212482769735E7 1932 +6.103293023818601E7 1933 +6.1032927141726024E7 1934 +3.709021264904225E7 1935 +2.7682551841429014E7 1936 +2.1273498978338182E7 1937 +4.597354917993289E7 1938 +1.2202787988136979E7 1939 +2.7682562489719376E7 1940 +2.1273500736469682E7 1941 +2.1273502567655977E7 1942 +5.759962752514529E7 1943 +6.103293867140644E7 1944 +6.103292666367206E7 1945 +5.759963362654527E7 1946 +2.7682555364786316E7 1947 +2.768255170403791E7 1948 +4.597362996435788E7 1949 +6.103292682380043E7 1950 +2.768255179735974E7 1951 +2.7682564251631413E7 1952 +5.759962704311152E7 1953 +2.1273498606220923E7 1954 +5.114352441003671E7 1955 +5.759962750163061E7 1956 +32.275962017278786 1957 +3.709021277991683E7 1958 +4.597355014615094E7 1959 +2.1273498886070885E7 1960 +6.103292716680684E7 1961 +5.7599627390610054E7 1962 +9719023.060992025 1963 +28.32564253649855 1964 +3.7090213650879174E7 1965 +3.709021302213531E7 1966 +9719024.080612516 1967 +3.709021214843138E7 1968 +9719023.111280642 1969 +5.759962752414469E7 1970 +5.855564282660682 1971 +259.34730658743376 1972 +2.1273499480681784E7 1973 +1.220273355082076E7 1974 +5.7599627133044474E7 1975 +2.7682552608788908E7 1976 +4.597354925135181E7 1977 +6.1032927012281895E7 1978 +5.759962706288306E7 1979 +3.7090212318375394E7 1980 +6.484430188925243 1981 +5.759962701860569E7 1982 +3.709021204848349E7 1983 +5.1143521056398965E7 1984 +3.709021309432352E7 1985 +3.7090212463773854E7 1986 +2.7682552497517373E7 1987 +5.114352164118063E7 1988 +9719023.222297085 1989 +3.709021310202785E7 1990 +5.1143525628951974E7 1991 +2.7682551636961937E7 1992 +5.085552330446811 1993 +11.930070856759638 1994 +1.2202733225840367E7 1995 +2.7682554383421347E7 1996 +9719023.295338295 1997 +3.709021353902029E7 1998 +9719023.347256232 1999 +3.70902132201577E7 2000 +5.759964285722492E7 2001 +3.709021216699278E7 2002 +9719023.47737221 2003 +3.709021238172742E7 2004 +1.2202733341425668E7 2005 +3.709021281719094E7 2006 +5.7599627230658E7 2007 +2.768257879501585E7 2008 +6.1032927507895015E7 2009 +3.2020655151293873 2010 +6.1032926824869215E7 2011 +9719023.237465985 2012 +3.709021289692289E7 2013 +2.768255184153306E7 2014 +2.1273499220566932E7 2015 +2.1273499623342898E7 2016 +4.597354888393054E7 2017 +17.453952587139916 2018 +5.759962736580299E7 2019 +4.597354929108098E7 2020 +3.709021277506812E7 2021 +2.1273498460543074E7 2022 +2.768255162229885E7 2023 +2.768255205969713E7 2024 +49.7959143425039 2025 +3.7090212502951205E7 2026 +5.114352104920896E7 2027 +2.7682552406143706E7 2028 +6.103292776703676E7 2029 +9719023.065665329 2030 +1.220273359023179E7 2031 +1.2202733274358923E7 2032 +6.103292723074884E7 2033 +5.7599627165346E7 2034 +5.114352116783586E7 2035 +2.7682552638791144E7 2036 +2.1273499591453996E7 2037 +11.671526798078952 2038 +7.407517390752409 2039 +5.7599627242265485E7 2040 +1.220273858551462E7 2041 +4.597354893356828E7 2042 +6.103292719482669E7 2043 +2.768255417786631E7 2044 +5.759962714790298E7 2045 +2.1273499085242208E7 2046 +1.2202734159877041E7 2047 +1.2202736013518324E7 2048 +1.2202733330469066E7 2049 +5.114352237445885E7 2050 +9719023.10056753 2051 +2.7682552836322665E7 2052 +2.76825520916238E7 2053 +5.7599627373013854E7 2054 +9719023.33209785 2055 +3.709022334625239E7 2056 +3.709021271622267E7 2057 +3.70902133615392E7 2058 +3.709021473002797E7 2059 +2.7682554072580535E7 2060 +5.75996276306647E7 2061 +3.709021304778376E7 2062 +5.114352724595313E7 2063 +4.061921926507434 2064 +6.103292955332802E7 2065 +3.709021216964222E7 2066 +2.7682552227673307E7 2067 +1.2202734282913895E7 2068 +4.5973549468213655E7 2069 +3.7090215154884554E7 2070 +5.759962771833174E7 2071 +1.2202734186303275E7 2072 +3.709021218204825E7 2073 +5.759962706054017E7 2074 +2.768255236352025E7 2075 +6.10329266133898E7 2076 +2.1273499038774822E7 2077 +3.709021315132673E7 2078 +5.759962717223394E7 2079 +1.2202733603137938E7 2080 +4.597354895198121E7 2081 +5.759962711592459E7 2082 +1.2202732973198563E7 2083 +4.597354890825592E7 2084 +5.2864700055649845 2085 +3.7090212558195636E7 2086 +1.2202734027512042E7 2087 +6.10329317431414E7 2088 +3.709021213908185E7 2089 +1.22027335849269E7 2090 +5.759962763656154E7 2091 +4.597354880207521E7 2092 +2.7682552544706967E7 2093 +2.1273500531922642E7 2094 +5.7599627190091334E7 2095 +3.7090212356045395E7 2096 +9719023.239894027 2097 +2.1273498463419773E7 2098 +10.966448925359561 2099 +5.759962890767333E7 2100 +2.768255249887704E7 2101 +16.576849477940147 2102 +4.5973549371464856E7 2103 +16.803590593992475 2104 +6.1032926847604714E7 2105 +6.1032928768131986E7 2106 +1.2202733187216524E7 2107 +3.709021635049309E7 2108 +4.5973549303651065E7 2109 +5.7599628106626E7 2110 +1.2202733574167278E7 2111 +2.768255480918226E7 2112 +9719023.333714264 2113 +4.597355166182501E7 2114 +2.7682552467260476E7 2115 +2.127349833794459E7 2116 +5.759962729788614E7 2117 +1.220273330637308E7 2118 +9719023.370993666 2119 +6.103292733575866E7 2120 +4.5973549414711E7 2121 +7.538841431084922 2122 +3.7090214530803785E7 2123 +5.114352093329215E7 2124 +5.7599627084380984E7 2125 +6.103292720565935E7 2126 +5.114352325438058E7 2127 +5.759962707468636E7 2128 +1.2202734022172652E7 2129 +6.103292660224819E7 2130 +5.114352090856188E7 2131 +2.127349883839534E7 2132 +5.114352080137228E7 2133 +1.2202736403290227E7 2134 +5.7599627376748845E7 2135 +70.9351722236787 2136 +6.1032926653723985E7 2137 +5.114352068740098E7 2138 +1.2202735169217577E7 2139 +3.709022247653895E7 2140 +6.103292702929993E7 2141 +3.709021310755002E7 2142 +6.103292810580551E7 2143 +3.7090212151874326E7 2144 +2.1273498700966988E7 2145 +6.103292768055578E7 2146 +1.2202735752907924E7 2147 +5.7599627063995466E7 2148 +3.709021412957874E7 2149 +3.709021248051717E7 2150 +4.597354918699414E7 2151 +4.5973549168999135E7 2152 +2.1273504687534854E7 2153 +5.114352064983364E7 2154 +1.2202732987116147E7 2155 +2.127349884639841E7 2156 +4.5973549148204826E7 2157 +4.5973549418677695E7 2158 +6.1032927637183964E7 2159 +2.7682552078991015E7 2160 +1.2202733730266126E7 2161 +9719023.15056657 2162 +4.597354956990751E7 2163 +3.709021263729715E7 2164 +3.709022265598598E7 2165 +2.768255182239203E7 2166 +5.11435208554381E7 2167 +2.7682551951729763E7 2168 +1.22027335640422E7 2169 +2.768255189003446E7 2170 +4.597355399814271E7 2171 +2.1273670310454648E7 2172 +3.709021224273325E7 2173 +4.597354894775379E7 2174 +1.2202735324574605E7 2175 +2.127349961105173E7 2176 +6.103292662343742E7 2177 +2.1273499238930535E7 2178 +9719023.493561756 2179 +4.597354902432101E7 2180 +2.12735035809982E7 2181 +4.5973549085152976E7 2182 +5.1143520850539565E7 2183 +5.114352073074201E7 2184 +3.709021618847292E7 2185 +5.1143521986417994E7 2186 +3.709021235210835E7 2187 +2.768255159052288E7 2188 +2.768255165540972E7 2189 +1.2202733741066612E7 2190 +9719023.177434994 2191 +6.10329300843384E7 2192 +4.597354901021552E7 2193 +5.759962794085121E7 2194 +6.103292717863412E7 2195 +9719024.351693869 2196 +4.52246004864695 2197 +56.156849982234135 2198 +2.7682552313989755E7 2199 +6.103292777480138E7 2200 +5.114352056276604E7 2201 +2.1273498619602725E7 2202 +2.1273501764351077E7 2203 +5.7599627212145E7 2204 +1.2202734244469648E7 2205 +3.709021244504016E7 2206 +9719024.179363407 2207 +3.7090213651449986E7 2208 +3.709021265969775E7 2209 +4.597355927754995E7 2210 +9719023.120196188 2211 +2.1273498741117045E7 2212 +9719023.240744302 2213 +6.103292799315093E7 2214 +5.1143521126875214E7 2215 +3.1369527525214256 2216 +1.2202734431001488E7 2217 +4.597354988434318E7 2218 +5.114352104552655E7 2219 +1.2202742730436577E7 2220 +3.709021282757187E7 2221 +3.7090212800535195E7 2222 +14.546966455445416 2223 +12.918773871018466 2224 +2.1273498344333913E7 2225 +5.11435210773106E7 2226 +6.103292710654274E7 2227 +6.103292696946455E7 2228 +6.103292668769855E7 2229 +3.7090214097952835E7 2230 +2.7682551609844197E7 2231 +6.103292890521435E7 2232 +1.2202733674463049E7 2233 +5.114352080737029E7 2234 +27.169554353916563 2235 +2.7682551846690327E7 2236 +2.7682552694151144E7 2237 +2.1273498651008125E7 2238 +6.103292746487824E7 2239 +6.103292650878492E7 2240 +2.1273498890299935E7 2241 +2.1273504432402834E7 2242 +2.768255346685134E7 2243 +5.759962738238326E7 2244 +2.7682551700548112E7 2245 +3.709021213652933E7 2246 +6.1301001937091115 2247 +2.1273498811587255E7 2248 +9719023.159117384 2249 +4.597354935208519E7 2250 +4.59735490480593E7 2251 +2.1273499454488683E7 2252 +9719023.094604192 2253 +16.78801323315497 2254 +2.1273498775175106E7 2255 +6.1032927160608545E7 2256 +4.597354884646769E7 2257 +1.2202732926397705E7 2258 +5.759962803995827E7 2259 +9719023.51685958 2260 +1.2202734699676707E7 2261 +4.5973549392963454E7 2262 +5.114352057397954E7 2263 +2.7682555044508062E7 2264 +4.5973549503946595E7 2265 +2.1273498598981246E7 2266 +3.7090212238095E7 2267 +7.400433596458607 2268 +189.82897085921712 2269 +4.597354926068423E7 2270 +1.2202734174630474E7 2271 +2.7682553227928463E7 2272 +12.78576345225408 2273 +17.45190543422693 2274 +8.449817618627819 2275 +19.34821818026452 2276 +2.1273498372362845E7 2277 +14.774390003262486 2278 +3.7090212858577676E7 2279 +2.1273498375816595E7 2280 +3.709021405029793E7 2281 +2.1273499083537936E7 2282 +6.1032952974320225E7 2283 +2.7682559075222407E7 2284 +1.220273443897554E7 2285 +2.1273499035178214E7 2286 +2.127349855121225E7 2287 +8.625700145022957 2288 +2.127349873247507E7 2289 +4.858701298644918 2290 +1.2202733679045333E7 2291 +2.1273499396822836E7 2292 +5.1143520814659454E7 2293 +5.114352056366066E7 2294 +3.951662108962669 2295 +6.103294772958917E7 2296 +6.103292741046837E7 2297 +9719024.247585585 2298 +2.1273504353858292E7 2299 +9719023.563410562 2300 +6.1032926874564916E7 2301 +2.7682554216234446E7 2302 +9719023.553385189 2303 +2.768255236948492E7 2304 +29.127799056328506 2305 +5.7599627091177456E7 2306 +1.2202733272423847E7 2307 +5.759962789141018E7 2308 +1.2202733043647947E7 2309 +5.7599627139013216E7 2310 +1.2202734199075077E7 2311 +2.7682564732153248E7 2312 +2.7682564617842432E7 2313 +3.709021257547009E7 2314 +4.6827571003101 2315 +2.1273498568981692E7 2316 +4.597355001269753E7 2317 +4.1859957361386275 2318 +2.1273498661932405E7 2319 +4.59735493376186E7 2320 +1.2202734140996404E7 2321 +4.597354935290878E7 2322 +1.2202736900995038E7 2323 +5.7599628098010175E7 2324 +2.127349896904381E7 2325 +5.114352309458159E7 2326 +9719023.270428035 2327 +5.114352143876267E7 2328 +9719023.316733407 2329 +5.759962748619592E7 2330 +9719023.560075127 2331 +4.019513742580287 2332 +1.2202733582040265E7 2333 +4.597354922239731E7 2334 +5.759962703691117E7 2335 +5.953232209656215 2336 +4.597355089222411E7 2337 +6.1032927560843445E7 2338 +1.2202733026566148E7 2339 +2.1273499660974268E7 2340 +1.2202733209849864E7 2341 +5.759963226836012E7 2342 +2.127349835277594E7 2343 +6.1766805749336 2344 +11.641278375006854 2345 +2.1273499043956496E7 2346 +5.759962729866167E7 2347 +3.709021325827266E7 2348 +5.114352107594984E7 2349 +2.1273498806657247E7 2350 +1.2202733626048895E7 2351 +2.127349914268025E7 2352 +4.5973549171195515E7 2353 +1.2202733592141341E7 2354 +5.759962732621665E7 2355 +332.9097057193106 2356 +2.768255175452166E7 2357 +2.127349850405879E7 2358 +7.6369916873504256 2359 +2.768255215715485E7 2360 +10.024544979844487 2361 +9719023.31706194 2362 +4.597354913298815E7 2363 +5.114352039760545E7 2364 +6.1032927272509284E7 2365 +5.114352176964622E7 2366 +5.1143520367976494E7 2367 +6.1032929014967576E7 2368 +2.1273510739525534E7 2369 +1.2202734629381923E7 2370 +5.114352045608866E7 2371 +4.116316096399547 2372 +6.103292950460457E7 2373 +2.7682551696392883E7 2374 +206.5255781383628 2375 +6.1032930864501245E7 2376 +1.2202733431587314E7 2377 +2.7682554233646106E7 2378 +3.709021228693455E7 2379 +3.7090213529033326E7 2380 +5.114352108732787E7 2381 +2.127349918745451E7 2382 +5.7599629404633574E7 2383 +3.709021314891871E7 2384 +757.6027606745744 2385 +3.70902127730869E7 2386 +5.759962752239452E7 2387 +1.220273297446056E7 2388 +2.768255255290985E7 2389 +3.075755563400377 2390 +5.75996276661737E7 2391 +4.59735489942815E7 2392 +2.7682551919774663E7 2393 +3.709021305956157E7 2394 +6.103292745364613E7 2395 +5.759962709491318E7 2396 +5.114352059228712E7 2397 +5.1143522697032504E7 2398 +5.759962703255975E7 2399 +5.759962728740126E7 2400 +5.114352104671181E7 2401 +4.597355267073516E7 2402 +2.7682553293686707E7 2403 +5.75996272916052E7 2404 +5.759963325967754E7 2405 +4.5973548941060625E7 2406 +6.1032927305108875E7 2407 +59.08133051517105 2408 +5.75996270856901E7 2409 +9719023.633342404 2410 +8.499153626559641 2411 +5.759962735862731E7 2412 +53.12442609412392 2413 +5.7599627845190965E7 2414 +3.709021425013409E7 2415 +9719023.124158366 2416 +4.597354943158006E7 2417 +2.768255214617842E7 2418 +6.103292688433937E7 2419 +5.1143524267257005E7 2420 +5.11435216300127E7 2421 +4.59735498521997E7 2422 +9719023.087535461 2423 +6.103292804368002E7 2424 +3.709021206600643E7 2425 +4.597354912341647E7 2426 +35.46088180957791 2427 +26.17766067777654 2428 +5.114352074469569E7 2429 +3.898284774460277 2430 +9719023.245860275 2431 +5.759962867249541E7 2432 +5.114352121967752E7 2433 +9719023.000476737 2434 +6.103293171443857E7 2435 +2.7682553128533695E7 2436 +2.7682558306457747E7 2437 +2.7682555469123963E7 2438 +2.7682552535651647E7 2439 +2.1273502472306635E7 2440 +2.12734983134574E7 2441 +9719023.089845533 2442 +9719023.217611505 2443 +4.597354902610277E7 2444 +5.114352114361825E7 2445 +9719023.072258472 2446 +9719023.617290558 2447 +1.2202733264163204E7 2448 +5.114352112838623E7 2449 +1.2202733587373126E7 2450 +2.127349875307023E7 2451 +5.114352088635547E7 2452 +1.220273291761299E7 2453 +5.3827966278912776 2454 +22.997417335063744 2455 +4.5973549581697196E7 2456 +4.597356046451444E7 2457 +4.597354917758781E7 2458 +9719023.02755474 2459 +2.1273500108140904E7 2460 +1.2202733207514515E7 2461 +5.114352077757271E7 2462 +6.103292682004005E7 2463 +2.1273498315042585E7 2464 +4.597354923138057E7 2465 +6.1032926547599874E7 2466 +9719024.179983463 2467 +5.759962714579675E7 2468 +3.709021318116029E7 2469 +23.642460454389724 2470 +5.759963079160462E7 2471 +9719023.488601062 2472 +9719023.33893189 2473 +5.114352093779036E7 2474 +5.759962719634997E7 2475 +6.1032999071780145E7 2476 +2.1273499680543527E7 2477 +9719023.089230504 2478 +5.759962746626303E7 2479 +4.5973548943045154E7 2480 +2.1273498597897634E7 2481 +9719023.47977643 2482 +2.7682551595576786E7 2483 +3.7090212382285185E7 2484 +1.2202733832640782E7 2485 +9719023.1316884 2486 +6.1032927766274974E7 2487 +3.7090212679825686E7 2488 +5.7599627195801884E7 2489 +3.709021383838614E7 2490 +5.114352044735456E7 2491 +2.768255203825622E7 2492 +5.7599627573106214E7 2493 +2.127349900481766E7 2494 +6.103292760854058E7 2495 +9719023.378458818 2496 +2.7682551767905265E7 2497 +5.11435214593541E7 2498 +4.5973549090681195E7 2499 +3.709021329993621E7 2500 +9719023.284340182 2501 +6.103292729986795E7 2502 +5.114352182158989E7 2503 +3.94018332539608 2504 +5.114355947228985E7 2505 +1.2202737692959856E7 2506 +5.114352078942369E7 2507 +5.1143522454799704E7 2508 +5.114352140439222E7 2509 +2.127349871202978E7 2510 +4.294051100872481 2511 +2.1273500252787013E7 2512 +5.7599628136761434E7 2513 +3.7090212543660425E7 2514 +6.1032927419786066E7 2515 +2.1273499912738726E7 2516 +9719023.557880595 2517 +26.001655231134126 2518 +6.103292761498176E7 2519 +2.7682552074747164E7 2520 +6.10329589559508E7 2521 +2.7682551548054527E7 2522 +12.614305925556188 2523 +4.597354899033346E7 2524 +6.103292769407342E7 2525 +4.597355194701766E7 2526 +5.759962736211925E7 2527 +5.1143521295138605E7 2528 +2.1273498960356183E7 2529 +3.7090212760706544E7 2530 +9719023.531305052 2531 +2.768255263480141E7 2532 +2.768255238526129E7 2533 +3.7090213762460746E7 2534 +9719023.266545089 2535 +1.2202734542150896E7 2536 +6.103292742038121E7 2537 +2.7682556015153036E7 2538 +2.7682552356763627E7 2539 +6.103292672479422E7 2540 +2.127349876636623E7 2541 +4.597354886929468E7 2542 +5.75996329269535E7 2543 +9719023.119556181 2544 +1.2202734165354034E7 2545 +29.95861415975999 2546 +2.7682551892628424E7 2547 +3.7090212175272256E7 2548 +9719023.5884642 2549 +6.103292749064753E7 2550 +33.508244727677834 2551 +6.103292766967655E7 2552 +6.570558885426761 2553 +5.759962749295788E7 2554 +4.597354885051664E7 2555 +26.259094484695506 2556 +2.1273499350850284E7 2557 +2.1273498706643183E7 2558 +4.597356982904325E7 2559 +3.709021318686341E7 2560 +2.1273498761534207E7 2561 +1.220274864575069E7 2562 +5.114352256758218E7 2563 +5.7599627967802085E7 2564 +3.709021368943475E7 2565 +4.5973549245021746E7 2566 +9719023.962297983 2567 +9719023.740461668 2568 +5.114352225063334E7 2569 +23.666768998775346 2570 +9719023.259098254 2571 +5.75996273009696E7 2572 +5.1143524620904796E7 2573 +2.1273499357811194E7 2574 +3.709021238319102E7 2575 +4.5973576995278515E7 2576 +1.2202733018557178E7 2577 +2.1273498386128064E7 2578 +2.7682552128525995E7 2579 +5.114352094207352E7 2580 +9719023.091187824 2581 +1.2202733922443246E7 2582 +3.709021325311518E7 2583 +2.7682552428994097E7 2584 +2.7682552013697755E7 2585 +5.759962905081806E7 2586 +4.597354979779089E7 2587 +9719023.052603284 2588 +1.2202735362238782E7 2589 +5.759962710275315E7 2590 +5.818839939469079 2591 +8.345893032768869 2592 +6.103292664874919E7 2593 +4.597355009292535E7 2594 +2.1273498669936206E7 2595 +33.97863196848964 2596 +3.709021510989249E7 2597 +18.152954945019715 2598 +5.11435206012935E7 2599 +5.759962733877696E7 2600 +3.7090213149108924E7 2601 +5.7599628448343344E7 2602 +6.103292776270164E7 2603 +1.2202733962212585E7 2604 +3.709021616305897E7 2605 +9719023.499905573 2606 +3.709021255747376E7 2607 +6.1032927411135174E7 2608 +6.1032926546706796E7 2609 +2.768255561459372E7 2610 +2.127352323473424E7 2611 +6.103292764515506E7 2612 +2.768255189901467E7 2613 +4.5973549106334835E7 2614 +5.056277213058789 2615 +4.5973549296942025E7 2616 +5.759962740529708E7 2617 +2.768255240032905E7 2618 +5.7599628727759995E7 2619 +5.759962925360862E7 2620 +9719023.31090824 2621 +2.1273498722094573E7 2622 +1.2202732865536148E7 2623 +6.103292780618588E7 2624 +2.1273498613637548E7 2625 +2.7682552611267857E7 2626 +9719023.158890253 2627 +2.1273499306092158E7 2628 +2.768255178734836E7 2629 +4.597354979696503E7 2630 +2.76825518701889E7 2631 +3.709021241338873E7 2632 +3.709021301382875E7 2633 +1.220273297751656E7 2634 +2.1273498720164735E7 2635 +5.759962738234418E7 2636 +2.1273498708723433E7 2637 +6.10329273575275E7 2638 +4.597354947500216E7 2639 +1.2202733262732716E7 2640 +2.127349976354972E7 2641 +3.70902121645748E7 2642 +6.103292717438245E7 2643 +9719024.498576516 2644 +2.127349896107853E7 2645 +5.759962723286026E7 2646 +2.127349964941395E7 2647 +1.2202734708748102E7 2648 +3.709021298222908E7 2649 +3.709021259759E7 2650 +1.2202734954383709E7 2651 +6.103292697492164E7 2652 +9719023.565875381 2653 +2.768753272263346E7 2654 +2.1273498932755433E7 2655 +3.709021274364806E7 2656 +5.7599627363573216E7 2657 +2.7682551550051566E7 2658 +4.597355030606032E7 2659 +6.10329265759989E7 2660 +1.2202734142608618E7 2661 +2.127349847958343E7 2662 +2.7682552385536145E7 2663 +9719023.196777739 2664 +2.7682552152198434E7 2665 +2.7682551577232942E7 2666 +4.5973549249148995E7 2667 +5.114352425315618E7 2668 +1.2202733705597186E7 2669 +1.2202734574700123E7 2670 +6.1032927266683355E7 2671 +4.597354961294637E7 2672 +1.2202732952967605E7 2673 +2.7682553967587706E7 2674 +6.103293029251681E7 2675 +6.10329492627418E7 2676 +3.709021275642693E7 2677 +9719023.222096667 2678 +5.7599627343333125E7 2679 +4.597355069067522E7 2680 +5.759962720690247E7 2681 +5.7599627370034695E7 2682 +42.80562721109435 2683 +1.220273572778404E7 2684 +5.7599627658439316E7 2685 +2.7682551623910896E7 2686 +3.709021405538127E7 2687 +1.2202733942024235E7 2688 +1.2202732902516192E7 2689 +4.597354938649662E7 2690 +2.1273498808585204E7 2691 +4.597355129266949E7 2692 +6.103293025680779E7 2693 +5.759962958800765E7 2694 +4.597354922431331E7 2695 +3.7090213341655366E7 2696 +2.7682552196508106E7 2697 +5.75996272586026E7 2698 +6.103292827333049E7 2699 +5.114352232004645E7 2700 +2.127349997414686E7 2701 +2.768255165109559E7 2702 +2.7682552936601885E7 2703 +6.1032934704086706E7 2704 +2.7682552083956823E7 2705 +4.597354944150202E7 2706 +2.7682553430675067E7 2707 +2.127349844398961E7 2708 +4.597354932741112E7 2709 +2.76825561523651E7 2710 +5.114352111327789E7 2711 +5.114352225638579E7 2712 +9719023.275306957 2713 +4.597354932336893E7 2714 +5.759962746138714E7 2715 +3.70902127264092E7 2716 +2.768255354060904E7 2717 +5.759962710065492E7 2718 +4.597354902886667E7 2719 +5.759962735535949E7 2720 +5.114352039187322E7 2721 +7.0618042395692875 2722 +6.103292888374076E7 2723 +4.597354899519055E7 2724 +5.1143520731198676E7 2725 +5.7599627176250465E7 2726 +3.709021238374608E7 2727 +4.597354886745325E7 2728 +6.103292744791042E7 2729 +9719023.97599059 2730 +2.7682553482688304E7 2731 +2.768257049691516E7 2732 +1.220273382471244E7 2733 +3.7090212093443215E7 2734 +2.12734987268513E7 2735 +5.114352109605895E7 2736 +5.11435223872268E7 2737 +2.7682553142250814E7 2738 +3.709021215798979E7 2739 +1.220273355024507E7 2740 +3.7090213614818044E7 2741 +1.220273360261392E7 2742 +5.759962705650744E7 2743 +4.5973549646374404E7 2744 +15.441796131170513 2745 +2.1273498435397793E7 2746 +5.1143521050661735E7 2747 +5.114352072579317E7 2748 +9719023.28910935 2749 +5.1143530691126734E7 2750 +9719023.00040859 2751 +4.5973549625207596E7 2752 +2.1273512778574225E7 2753 +6.103292724406875E7 2754 +9719023.11016955 2755 +2.1273501151578415E7 2756 +5.759962834888638E7 2757 +2.1273499029087108E7 2758 +9719023.14276774 2759 +3.709021501432199E7 2760 +6.1032927384360954E7 2761 +5.114352283542237E7 2762 +9719023.349091025 2763 +3.709021245641768E7 2764 +5.759962704469684E7 2765 +2.768255235342237E7 2766 +5.759962721041102E7 2767 +5.7599627097279444E7 2768 +2.768255253457125E7 2769 +2.768255172926992E7 2770 +3.709021219111476E7 2771 +6.103292679552911E7 2772 +2.1273501386504814E7 2773 +4.597354929506069E7 2774 +1.2202733310080925E7 2775 +5.759962727620957E7 2776 +1.2202766255727872E7 2777 +9719023.191026792 2778 +6.1032927481574506E7 2779 +5.759962721486547E7 2780 +6.103293063117062E7 2781 +5.759962727571487E7 2782 +65.37151412632087 2783 +5.759962742918163E7 2784 +2.768255198706378E7 2785 +1.2202733577546086E7 2786 +5.114352092528161E7 2787 +9719023.819490353 2788 +7.637770460071739 2789 +5.99411686990649 2790 +5.1143520372748315E7 2791 +4.597354993788647E7 2792 +4.59735492954791E7 2793 +2.7682554763056364E7 2794 +2.7682554086214162E7 2795 +94.63541807959065 2796 +1.220273378209418E7 2797 +2.7682553731369823E7 2798 +6.1032927050999746E7 2799 +9719023.031740416 2800 +16.465436498650416 2801 +7.232879672193539 2802 +6.103292675951425E7 2803 +2.1273500902633797E7 2804 +3.709021310912327E7 2805 +2.7682552770308774E7 2806 +2.7682551848470114E7 2807 +9719023.269314285 2808 +9719023.247287273 2809 +9719027.869469361 2810 +19.061064715978485 2811 +5.1143520847573124E7 2812 +9719023.230341576 2813 +4.597425197274584E7 2814 +1.22027338416436E7 2815 +5.1143520467640705E7 2816 +6.1032927250766456E7 2817 +2.1273501415594622E7 2818 +1.22027336719354E7 2819 +2.127349906278983E7 2820 +3.709021233748997E7 2821 +3.709021272918604E7 2822 +3.709021208949124E7 2823 +1.2202735581577044E7 2824 +5.114360248006832E7 2825 +1.2202733253455397E7 2826 +6.103292728899886E7 2827 +5.759962752370159E7 2828 +5.114352062850723E7 2829 +2.127351173164042E7 2830 +2.7682554175643366E7 2831 +5.114352069171196E7 2832 +3.7090212269703E7 2833 +5.114352117063405E7 2834 +2.76825561482429E7 2835 +2.7682552030574664E7 2836 +2.127350013266024E7 2837 +5.759962712882283E7 2838 +18.2166485212859 2839 +2.768255237907269E7 2840 +3.709021265811458E7 2841 +4.597354879691873E7 2842 +4.597354932023345E7 2843 +1.220273331787953E7 2844 +9719025.011093432 2845 +4.597354882100116E7 2846 +5.759962768012717E7 2847 +5.759962997897681E7 2848 +13.899725262794817 2849 +6.1032928043875515E7 2850 +2.1273498786821306E7 2851 +1.2202732964019809E7 2852 +9719023.197836163 2853 +2.7682552201498315E7 2854 +2.7682551593981653E7 2855 +2.127349833552119E7 2856 +5.759962835822503E7 2857 +2.7682553127809316E7 2858 +2.1273498899800524E7 2859 +6.1032926584642105E7 2860 +2.7682553178359527E7 2861 +1.220273529743882E7 2862 +2.127349898313935E7 2863 +10.218077844841757 2864 +6.103292651604943E7 2865 +5.759962723236681E7 2866 +5.759962739996447E7 2867 +9719023.05186733 2868 +1.22027341671093E7 2869 +5.759962722050794E7 2870 +5.75996279622787E7 2871 +5.1143525618984185E7 2872 +2.127349857183394E7 2873 +6.1032927108628646E7 2874 +2.768255207676277E7 2875 +3.709021268735606E7 2876 +1.2202733154389888E7 2877 +4.597355116733044E7 2878 +6.103292828865077E7 2879 +5.1143521584205754E7 2880 +1.2202802005572733E7 2881 +3.709021238293876E7 2882 +2.7682779602541327E7 2883 +2.768264288638258E7 2884 +5.7599627473251216E7 2885 +5.1143520817113616E7 2886 +4.5973549874789596E7 2887 +6.103292707893465E7 2888 +3.709021234625485E7 2889 +4.5973549233493015E7 2890 +1.220273313367394E7 2891 +5.1143520502545916E7 2892 +4.59735490811037E7 2893 +4.597354918638379E7 2894 +2.127349849503444E7 2895 +2.7682552913117882E7 2896 +6.1032927503668815E7 2897 +4.597354905093013E7 2898 +5.114352047634473E7 2899 +4.597355059091107E7 2900 +4.5973549026850104E7 2901 +3.709021361863357E7 2902 +3.709021277333713E7 2903 +5.759962708882589E7 2904 +4.59735491652005E7 2905 +5.114352066608989E7 2906 +2.7682552024781402E7 2907 +2.7682553185990162E7 2908 +5.759962724987006E7 2909 +5.1143521398699544E7 2910 +2.127352391626688E7 2911 +2.7682552056820504E7 2912 +3.709021467493171E7 2913 +4.5973549068878815E7 2914 +4.5973549814109206E7 2915 +2.7682552606726136E7 2916 +5.7599627118166454E7 2917 +1.220273592530424E7 2918 +6.1032927355345316E7 2919 +5.114352076314446E7 2920 +5.114352070677329E7 2921 +2.7682552010552462E7 2922 +3.7090213632021174E7 2923 +142.22779397958374 2924 +5.759963707338325E7 2925 +2.1273498384128537E7 2926 +5.759962725151361E7 2927 +6.103292750535879E7 2928 +6.882572737082159 2929 +5.7599628011393264E7 2930 +2.7682551701084338E7 2931 +1.2202733963359281E7 2932 +3.709021240762441E7 2933 +8.613892533410635 2934 +2.1273498461251386E7 2935 +4.597354923442649E7 2936 +1.2202733468207845E7 2937 +4.597354977932568E7 2938 +5.1143520995721914E7 2939 +34.272945991089394 2940 +6.103292709823343E7 2941 +2.768255426275691E7 2942 +5.759962746470978E7 2943 +5.759962756412602E7 2944 +4.5973548928216256E7 2945 +18.319800031757627 2946 +2.127349858212046E7 2947 +2.7682552015516635E7 2948 +5.114352045910298E7 2949 +5.114352061137846E7 2950 +4.597355016914207E7 2951 +5.7599627362646624E7 2952 +2.768255277258839E7 2953 +204.59901349392032 2954 +4.597354902562865E7 2955 +2.1273516638751864E7 2956 +6.1032927060675845E7 2957 +2.7682553325961486E7 2958 +3.709021283139264E7 2959 +1.2202733414676776E7 2960 +1.2202736311879145E7 2961 +6.103292787147993E7 2962 +5.1143521705676526E7 2963 +4.5973549392528154E7 2964 +2.12734992550328E7 2965 +4.59735492113652E7 2966 +6.103292746098259E7 2967 +9719033.155386338 2968 +2.1273498348982714E7 2969 +9719023.74756286 2970 +3.7090212391196236E7 2971 +2.7682552629474845E7 2972 +9719023.615821771 2973 +36.00076232352434 2974 +24.243107415370286 2975 +5.1143520694301516E7 2976 +4.5973558746702455E7 2977 +5.114352053329278E7 2978 +9719023.667479247 2979 +5.759962754051302E7 2980 +3.709021339089507E7 2981 +2.1273498949983656E7 2982 +6.103292699549869E7 2983 +23.28877771705592 2984 +4.597354906124324E7 2985 +2.1273499365775242E7 2986 +4.597354941744993E7 2987 +3.70902122686932E7 2988 +2.1273505036064498E7 2989 +9719023.2586709 2990 +2.1273498415853888E7 2991 +6.572321920465852 2992 +5.7599627449577846E7 2993 +1.2202734672769152E7 2994 +5.759962704885162E7 2995 +11.44053344804028 2996 +2.1273498552026574E7 2997 +1.2202734236426922E7 2998 +6.103292751744878E7 2999 +1.2202733612084927E7 3000 +6.103292687570687E7 3001 +5.7599627481656946E7 3002 +6.1032927576900855E7 3003 +5.759963246214735E7 3004 +5.1143526482844226E7 3005 +2.7682552599689137E7 3006 +2.127350008714252E7 3007 +3.709021358381772E7 3008 +2.7682551871180315E7 3009 +88.57270511038902 3010 +9719023.53606901 3011 +2.7682554469859287E7 3012 +5.114352060465321E7 3013 +33.845834142424906 3014 +6.1032927567982815E7 3015 +6.1032927094739676E7 3016 +3.7090248352257945E7 3017 +1.2202733077984992E7 3018 +5.759962743084392E7 3019 +4.597354901330018E7 3020 +6.176711596930998 3021 +5.75996273646579E7 3022 +1.220273376806545E7 3023 +3.709021305186631E7 3024 +5.114352066060965E7 3025 +4.597354994704564E7 3026 +3.709021367883046E7 3027 +2.7682553549056284E7 3028 +2.7682553824030317E7 3029 +2.7682551770171534E7 3030 +9719023.296441799 3031 +4.5973548959650636E7 3032 +2.1273499648988366E7 3033 +2.1273505278465517E7 3034 +2.1273498549835697E7 3035 +3.709021305909341E7 3036 +4.59735492125534E7 3037 +6.855345738321888 3038 +9719023.22901755 3039 +2.1273499027695414E7 3040 +9719023.23575605 3041 +6.103292662235552E7 3042 +2.127350047330294E7 3043 +4.5973549337493815E7 3044 +5.114352153981848E7 3045 +5.759962754940046E7 3046 +6.103292706191598E7 3047 +4.59735490264287E7 3048 +6.1032926847009785E7 3049 +2.1273499082560815E7 3050 +5.114352057869126E7 3051 +4.597355016004444E7 3052 +5.759962887186719E7 3053 +2.127349934695791E7 3054 +5.114352200753144E7 3055 +3.709021226702479E7 3056 +41.3069921553035 3057 +1.2202732862476021E7 3058 +1.2202734358635455E7 3059 +5.114352037934443E7 3060 +4.225349865489846 3061 +5.114352062763985E7 3062 +1.2202733151890177E7 3063 +9719023.483134177 3064 +3.709021364610488E7 3065 +5.114352048227785E7 3066 +2.7682551831767958E7 3067 +1.2202736532792559E7 3068 +9719023.042406088 3069 +2.1273498775930542E7 3070 +16.84289135782095 3071 +1.2202733432381311E7 3072 +5.114352100190368E7 3073 +2.7682551959769074E7 3074 +9719042.132800627 3075 +9719023.388545403 3076 +2.7682552009142183E7 3077 +4.5973549171163805E7 3078 +5.7599627301644035E7 3079 +5.114352085944608E7 3080 +5.759962893628783E7 3081 +4.597355078351177E7 3082 +2.1273499422291476E7 3083 +6.103292721810242E7 3084 +9719023.072933985 3085 +3.709021423016927E7 3086 +5.7599627148209274E7 3087 +2.1273499029199883E7 3088 +6.1032927096546516E7 3089 +5.1143520807599686E7 3090 +3.709021263990416E7 3091 +3.70902125038964E7 3092 +1.2202734827037435E7 3093 +2.1273500122531682E7 3094 +9719023.305969812 3095 +2.12734986958056E7 3096 +4.597354905246818E7 3097 +6.103292757505132E7 3098 +9719023.333640404 3099 +9.234340296306884 3100 +5.759962763353383E7 3101 +3.7090231428187154E7 3102 +4.5973549163364865E7 3103 +6.103292662437734E7 3104 +2.7682552423097793E7 3105 +2.127349899985761E7 3106 +5.114352041189117E7 3107 +2.127349965314923E7 3108 +1.2202736043490373E7 3109 +2.1273498618942153E7 3110 +14.042545350007039 3111 +3.7090220146091715E7 3112 +4.5973549453354485E7 3113 +5.759963305768367E7 3114 +2.7682552679797933E7 3115 +2.7682551930923376E7 3116 +2.12734988023925E7 3117 +2.768255158519329E7 3118 +6.103292790115298E7 3119 +1.2202735715536855E7 3120 +2.127349831892798E7 3121 +2.7682552170281287E7 3122 +1.220273322871962E7 3123 +1.2202733367205491E7 3124 +1.2202733412203223E7 3125 +4.597354995018777E7 3126 +5.725482212189928 3127 +2.1273498652151465E7 3128 +9719023.725389956 3129 +30.937295964234583 3130 +3.7090213387399286E7 3131 +1.2202733889343865E7 3132 +6.1032932316449486E7 3133 +1.2202736160875902E7 3134 +6.103293847700629E7 3135 +4.597354879823859E7 3136 +4.597355082623502E7 3137 +2.7682555252665192E7 3138 +3.2825987064109547 3139 +2.7682552939613584E7 3140 +1.2202733049795449E7 3141 +4.597354951321752E7 3142 +1.220273307844478E7 3143 +5.759962708516241E7 3144 +4.5973550035870954E7 3145 +5.114352066481355E7 3146 +3.8278180567595115 3147 +9719023.096418284 3148 +1.2202734007924838E7 3149 +2.7682553242466513E7 3150 +5.114352054748948E7 3151 +5.1143520788212344E7 3152 +9719023.274074743 3153 +2.7682552837955695E7 3154 +4.597355005401063E7 3155 +6.1032927374018036E7 3156 +2.127349889179248E7 3157 +6.103292691129965E7 3158 +3.709021466568917E7 3159 +6.103292727075946E7 3160 +2.127349875743686E7 3161 +5.114352283843742E7 3162 +4.5973560868985236E7 3163 +6.1032941428163804E7 3164 +140.1344467363845 3165 +1.2202733937868329E7 3166 +5.114352072463306E7 3167 +9719023.209476419 3168 +9719023.167116484 3169 +5.75996273036344E7 3170 +1.2202733543345876E7 3171 +3.3057798141193886 3172 +9719023.03554769 3173 +4.76157219253336 3174 +2.768255192419603E7 3175 +5.759962743946747E7 3176 +5.1143568393007025E7 3177 +1.2202733568467416E7 3178 +9719023.248365011 3179 +6.037600766483973 3180 +2.7682551654171273E7 3181 +5.114352039158826E7 3182 +5.1143520662280604E7 3183 +3.709021239037052E7 3184 +6.103292664003222E7 3185 +2.127349851856003E7 3186 +6.103292729495142E7 3187 +2.7682552357363183E7 3188 +3.709022278058963E7 3189 +6.103292734491048E7 3190 +3.709021302128474E7 3191 +5.759963235726438E7 3192 +7.034239197309764 3193 +3.9753520496442905 3194 +2.768255302997056E7 3195 +2.1273499433446698E7 3196 +4.597354895492144E7 3197 +4.5973549204549834E7 3198 +4.65497672457592 3199 +1.2202735165052952E7 3200 +3.709021249349804E7 3201 +9719024.36594801 3202 +1.2202736559695862E7 3203 +4.330546420974733 3204 +1.2202732999847496E7 3205 +4.5973549086569E7 3206 +2.7682552585244797E7 3207 +2.768255206232436E7 3208 +2.1273500339117926E7 3209 +5.7599627455947116E7 3210 +6.103292674057081E7 3211 +5.114352223422518E7 3212 +36.00516939454013 3213 +6.103292739347218E7 3214 +2.768255269110992E7 3215 +9719023.08172829 3216 +9719023.398688748 3217 +2.1273498565215897E7 3218 +1.2202733304864807E7 3219 +5.759962715561055E7 3220 +2.7682551674779557E7 3221 +2.768255257203919E7 3222 +1.2202734833669651E7 3223 +5.114352138937998E7 3224 +20.11924899426219 3225 +1.2202734095852444E7 3226 +2.7682556829935476E7 3227 +5.7599627734979145E7 3228 +1.220273394221703E7 3229 +5.1143521206088506E7 3230 +2.768255532309404E7 3231 +5.114352090306212E7 3232 +3.709023624176148E7 3233 +1.2202734308770355E7 3234 +190.33703975371378 3235 +4.597354892244184E7 3236 +2.1273498339353472E7 3237 +2.7682552857109282E7 3238 +2.1273498755524322E7 3239 +5.759962943552443E7 3240 +4.597354921959044E7 3241 +5.1143520580198206E7 3242 +6.103292728424587E7 3243 +2.768255218725708E7 3244 +2.768255232561272E7 3245 +4.597354925920105E7 3246 +14.754883729254212 3247 +6.1032926615465045E7 3248 +304.13917464799874 3249 +2.1273498963779237E7 3250 +5.114352152514355E7 3251 +4.597354882203151E7 3252 +4.337197807952555 3253 +2.76825516455678E7 3254 +6.103292753330178E7 3255 +9719023.269114936 3256 +5.759962724401273E7 3257 +6.9448702668924485 3258 +6.103292872759308E7 3259 +5.7599627267951824E7 3260 +3.709021213380028E7 3261 +6.103292723129252E7 3262 +5.1143521583456926E7 3263 +4.5973549311143E7 3264 +2.768255390728445E7 3265 +5.114352074629547E7 3266 +2.768255222423847E7 3267 +5.114352102149051E7 3268 +1.220273822162786E7 3269 +1.2202736538749708E7 3270 +9719023.160672974 3271 +3.709021388234903E7 3272 +13.082995311230128 3273 +9719024.582622074 3274 +926.2305699648317 3275 +4.597354917015535E7 3276 +5.759962748064692E7 3277 +2.127349836044192E7 3278 +3.709021293919673E7 3279 +5.759962791244635E7 3280 +1.2202733451532105E7 3281 +5.114352143522898E7 3282 +5.1143521823124416E7 3283 +5.11435211613627E7 3284 +2.127349901410116E7 3285 +13.51321589383901 3286 +5.759962818809993E7 3287 +5.759963252854077E7 3288 +1.220273359192408E7 3289 +7.114240384745068 3290 +82.3715496411962 3291 +4.5973548954595074E7 3292 +8.003821828631784 3293 +5.114352090743151E7 3294 +4.5973549596294515E7 3295 +3.709021247836743E7 3296 +4.597354925851313E7 3297 +2.7682551579545353E7 3298 +4.597354941117731E7 3299 +4.597354900822778E7 3300 +8.248860426391984 3301 +9719023.17732973 3302 +3.709021644693076E7 3303 +2.127349847365354E7 3304 +9719023.332187502 3305 +9719023.561718902 3306 +2.1273503419367705E7 3307 +4.597354891070694E7 3308 +5.1143520731151976E7 3309 +6.10329280459275E7 3310 +4.597354926267678E7 3311 +5.75996277572142E7 3312 +9719025.070968265 3313 +5.114352089280503E7 3314 +3.70902123343543E7 3315 +1.2202735251629962E7 3316 +4.597354923405101E7 3317 +4.597354916829084E7 3318 +3.70902148730323E7 3319 +1.2202733081883507E7 3320 +9719023.404752133 3321 +5.7599627351749904E7 3322 +2.127349879704505E7 3323 +1.220273427004583E7 3324 +9719023.027896231 3325 +6.103292694137931E7 3326 +2.768255301194167E7 3327 +4.59735503840637E7 3328 +1.2202733069505125E7 3329 +4.597354927911126E7 3330 +5.114352080105129E7 3331 +9719023.099074068 3332 +1.2202734624920009E7 3333 +4.597354934396253E7 3334 +5.114352163760842E7 3335 +9719023.64185928 3336 +4.5973549560836636E7 3337 +5.7599627241579294E7 3338 +1.2202733437211005E7 3339 +2.127350646283128E7 3340 +3.7090213162942275E7 3341 +5.759971151394888E7 3342 +1.220273363936145E7 3343 +2.127349985686398E7 3344 +5.759964077983784E7 3345 +6.10329272528415E7 3346 +1.2202733829204727E7 3347 +5.759962765522469E7 3348 +6.1032927151325606E7 3349 +2.1273498529674444E7 3350 +4.597354887244632E7 3351 +5.1143521191309094E7 3352 +2.1273514822889082E7 3353 +1.2202734140530623E7 3354 +2.7682552218357503E7 3355 +4.597354906405276E7 3356 +4.597354938244558E7 3357 +4.943630732293084 3358 +28.990493647086872 3359 +5.1143520858500384E7 3360 +5.114352641189369E7 3361 +2.127349956100482E7 3362 +5.114352588766101E7 3363 +4.597354937130184E7 3364 +9719023.54412346 3365 +2.1273498396220755E7 3366 +5.759962705680339E7 3367 +5.7599627329419985E7 3368 +53.03531742364968 3369 +9719023.327337988 3370 +5.75996412324771E7 3371 +3.709021216279639E7 3372 +4.5973549305413485E7 3373 +3.709021301731827E7 3374 +1.2202733833397891E7 3375 +6.103292754004637E7 3376 +6.10329273495686E7 3377 +4.597354975724438E7 3378 +2.1273498765448898E7 3379 +3.7090212938666254E7 3380 +6.103292790210144E7 3381 +2.1273498610539652E7 3382 +2.768255224730239E7 3383 +3.709021459118027E7 3384 +1.2202733335128019E7 3385 +2.7682551558713045E7 3386 +9719023.192074222 3387 +2.1273498959607244E7 3388 +2.1273498481805407E7 3389 +9719031.64906973 3390 +2.1273500292330556E7 3391 +2.1273510598254893E7 3392 +1.2202732995724743E7 3393 +1.2202733395278072E7 3394 +4.597354934870432E7 3395 +6.103292682138306E7 3396 +5.7599627274824895E7 3397 +5.759962752435506E7 3398 +7.19507549715877 3399 +2.7682551804630786E7 3400 +1.2202734134762673E7 3401 +3.70902134077256E7 3402 +4.5973549311427176E7 3403 +2.1273500385874573E7 3404 +9.384242254658007 3405 +5.114352057539373E7 3406 +1.2202736401290223E7 3407 +6.10329273440516E7 3408 +6.103292693710196E7 3409 +2.1273499141015023E7 3410 +4.597355052301596E7 3411 +2.768255217089653E7 3412 +95.27124178914848 3413 +5.759962717102794E7 3414 +8.604589367621442 3415 +5.5057501376426865 3416 +2.1273499527508963E7 3417 +4.5973548817192964E7 3418 +4.597354990312037E7 3419 +6.103292680129778E7 3420 +20.306116909405105 3421 +3.709021563160195E7 3422 +10.876527641317976 3423 +2.768255286854423E7 3424 +5.759962777734293E7 3425 +3.709023320148783E7 3426 +2.127350644576928E7 3427 +3.709021313303079E7 3428 +6.103292711443688E7 3429 +3.709021326404337E7 3430 +2.1273499541954506E7 3431 +1.2202740756547406E7 3432 +6.103292784777496E7 3433 +5.759962789363946E7 3434 +3.4119560570193856 3435 +89.01327264893179 3436 +1.2202734803002348E7 3437 +4.5973550391240716E7 3438 +3.709021217771506E7 3439 +25.598066790912398 3440 +9719023.41282983 3441 +3.7090216973891646E7 3442 +5.114352203667018E7 3443 +6.103292656874812E7 3444 +16.396047922226817 3445 +9719023.345048433 3446 +6.103292943737722E7 3447 +5.114352094403634E7 3448 +2.768255434721228E7 3449 +3.709021337158541E7 3450 +4.597354966929685E7 3451 +5.7599627353098206E7 3452 +5.759962741461028E7 3453 +4.597355196392083E7 3454 +5.848337661333651 3455 +1.2202733651097935E7 3456 +3.709021225810698E7 3457 +1.2202735635911113E7 3458 +5.7599628836733796E7 3459 +6.103292653068901E7 3460 +2.127349840052129E7 3461 +2.7682555128255032E7 3462 +5.114352084292897E7 3463 +2.7682566657829147E7 3464 +3.709021324989094E7 3465 +6.103294226943807E7 3466 +2.1273498931038395E7 3467 +2.7682551964318685E7 3468 +9719023.262803867 3469 +31.763276128648474 3470 +5.114352104954706E7 3471 +6.10329268209418E7 3472 +9719023.321564352 3473 +36.0698356467051 3474 +5.7599630084166564E7 3475 +5.759962729912915E7 3476 +1.220273462796151E7 3477 +5.759962702984326E7 3478 +5.759962710584484E7 3479 +5.759962705697561E7 3480 +6.103292819247619E7 3481 +2.7682554350039806E7 3482 +2.1273498880455207E7 3483 +5.759963289984367E7 3484 +5.114352053861504E7 3485 +3.709021285276831E7 3486 +4.5973549071100935E7 3487 +6.103292661633351E7 3488 +5.759962768748021E7 3489 +1.2202733766639234E7 3490 +19.13836739497108 3491 +5.114352060044825E7 3492 +6.103292748366319E7 3493 +3.70902134468119E7 3494 +5.7599627044930466E7 3495 +3.709021242549862E7 3496 +4.59735489510215E7 3497 +9719023.483940981 3498 +3.8082738412686075 3499 +2.127349832808704E7 3500 +4.470045952331832 3501 +4.597354882927701E7 3502 +9719024.135952465 3503 +4.597355073030154E7 3504 +2.768255300066241E7 3505 +3.709021482756175E7 3506 +6.815436226628803 3507 +4.5973549251820914E7 3508 +4.597354889215286E7 3509 +4.5973550561964884E7 3510 +2.127349892555596E7 3511 +3.709021230242608E7 3512 +2.1273500818541475E7 3513 +2.7682552007040136E7 3514 +5.114352111551515E7 3515 +6.103292695250397E7 3516 +4.056085126452852 3517 +274.32351714804525 3518 +5.114353956191654E7 3519 +5.1143521017446876E7 3520 +5.759962738718654E7 3521 +3.7090213120519795E7 3522 +2.1273498397530735E7 3523 +5.759962753814418E7 3524 +5.114353309758007E7 3525 +2.7682552584272973E7 3526 +5.114352051516605E7 3527 +3.709021472635921E7 3528 +2.1273523878579568E7 3529 +9719023.021386048 3530 +2.7682552398617383E7 3531 +1.2202733727051703E7 3532 +4.5973552214697175E7 3533 +17.159026664772107 3534 +5.1143520792786755E7 3535 +6.103292670490194E7 3536 +1.2202733274404317E7 3537 +5.7599627084517725E7 3538 +64.4784371139421 3539 +5.114352285109346E7 3540 +21.562197414000977 3541 +2.1273499676823944E7 3542 +6.103292687488402E7 3543 +4.597354916073379E7 3544 +6.10329280978352E7 3545 +6.103292686764392E7 3546 +1.2202734854895165E7 3547 +4.8122902746313985 3548 +3.7090212064944886E7 3549 +6.103292763614562E7 3550 +5.11435209970915E7 3551 +5.1143521508756004E7 3552 +3.7090212996554755E7 3553 +4.59735490442218E7 3554 +5.7599627366288126E7 3555 +2.1273499623637177E7 3556 +5.1143520868433796E7 3557 +1.2202734782549763E7 3558 +4.597355250066976E7 3559 +3.709021282972581E7 3560 +9719023.274986919 3561 +2.127349864907527E7 3562 +2.768255248912201E7 3563 +2.1273498683309413E7 3564 +3.7090212054938786E7 3565 +4.5973549190572746E7 3566 +2.7682552012015935E7 3567 +4.597354883520007E7 3568 +6.10329274011974E7 3569 +2.1273499277228918E7 3570 +2.7682553117163904E7 3571 +4.597354921967159E7 3572 +5.114352590308922E7 3573 +5.759962741530709E7 3574 +1.2202741810131239E7 3575 +5.759962806616973E7 3576 +6.10329287834964E7 3577 +2.1273498908689007E7 3578 +6.1032926662160225E7 3579 +1.220273342671027E7 3580 +9719023.288373671 3581 +3.709021246926263E7 3582 +5.759962835003608E7 3583 +2.127349833128513E7 3584 +1.2202734455562605E7 3585 +2.7682552621518716E7 3586 +5.114352052374538E7 3587 +3.709021358002492E7 3588 +2.127349846463457E7 3589 +5.114352067250681E7 3590 +5.114352053638781E7 3591 +5.114352116464748E7 3592 +2.1273499577943433E7 3593 +5.114352159137279E7 3594 +4.213039486435793 3595 +1.2202733566203019E7 3596 +4.597354899122254E7 3597 +9719023.57921335 3598 +6.1032926911033265E7 3599 +5.759962747176514E7 3600 +6.103292789971793E7 3601 +4.791294585428595 3602 +2.127350194372477E7 3603 +2.7682552369988427E7 3604 +5.759962817003422E7 3605 +9719023.182690421 3606 +5.114352073676201E7 3607 +5.759962736706156E7 3608 +6.1032926788329355E7 3609 +4.59735488524067E7 3610 +9719023.17511125 3611 +4.597354922417561E7 3612 +2.1273498608250227E7 3613 +5.114354162888357E7 3614 +1.220273435784251E7 3615 +2.1273498975580584E7 3616 +5.759962705267048E7 3617 +5.11435212096234E7 3618 +5.7599631603026904E7 3619 +5.114352038235143E7 3620 +4.597354927523152E7 3621 +6.103292667135483E7 3622 +5.1143520442227915E7 3623 +1.2202735940087363E7 3624 +4.5973549018752165E7 3625 +2.7682551829457246E7 3626 +4.597355315555237E7 3627 +5.1143521202982455E7 3628 +3.322159395221293 3629 +5.7599640201724105E7 3630 +5.114352147047962E7 3631 +3.709021292553572E7 3632 +1.2202733947033525E7 3633 +5.759962731159077E7 3634 +2.1273498705875933E7 3635 +5.114352037871122E7 3636 +1.2202748492651435E7 3637 +1.2202733811050275E7 3638 +1.2202733339599686E7 3639 +2.7682551678859882E7 3640 +1.2202737721213618E7 3641 +30.631548681226192 3642 +2.768255460376525E7 3643 +9719023.130701002 3644 +9719023.036545169 3645 +9719023.216376979 3646 +3.7090212360284664E7 3647 +4.597354918142963E7 3648 +69.59427823533724 3649 +1.2202733703256533E7 3650 +3.7090212249556005E7 3651 +5.1143547033450656E7 3652 +5.1143521058235124E7 3653 +3.709023150644439E7 3654 +2.7682553116401285E7 3655 +2.1273498855641607E7 3656 +6.103292908643853E7 3657 +2.7682551895278677E7 3658 +5.114352091174037E7 3659 +6.103293105939041E7 3660 +4.597354943436636E7 3661 +2.127349891806532E7 3662 +2.7682551885538157E7 3663 +9719023.140767837 3664 +5.1143520868637756E7 3665 +4.597354956732653E7 3666 +2.1273499123690784E7 3667 +2.127349962095768E7 3668 +3.709021218794955E7 3669 +5.759962727809019E7 3670 +2.7682552543023806E7 3671 +5.7599627251422785E7 3672 +9719023.42580943 3673 +2.127349941706654E7 3674 +2.768255361359595E7 3675 +5.114352137091061E7 3676 +1.2202734068483084E7 3677 +5.114352064286555E7 3678 +4.597354969872379E7 3679 +6.103292718948864E7 3680 +2.7682552167444684E7 3681 +9719023.396490829 3682 +9719023.310612777 3683 +5.759962707867572E7 3684 +3.709021255841604E7 3685 +4.279713148554999 3686 +64.47720169325959 3687 +4.5973555159508154E7 3688 +1.2202733699995171E7 3689 +1.2202733555380508E7 3690 +6.1032927035355784E7 3691 +2.7682552614009745E7 3692 +9719023.609999387 3693 +9719023.314429678 3694 +2.7682553519669767E7 3695 +2.1273498322078194E7 3696 +1.220274877347946E7 3697 +3.7090212954680525E7 3698 +1.2202733749005686E7 3699 +5.656809853337738 3700 +8.381641716689362 3701 +3.709021307850317E7 3702 +138.33811189061728 3703 +7.47719387994497 3704 +25.393373548307796 3705 +1.2202734619237656E7 3706 +4.597354907000811E7 3707 +5.7599627101789705E7 3708 +9719023.47409076 3709 +5.114352082569347E7 3710 +2.768255270536762E7 3711 +1.2202733835575E7 3712 +2.1273499530058816E7 3713 +5.759962801436391E7 3714 +5.504316451307913 3715 +6.10329274146177E7 3716 +6.068085122204758 3717 +3.7090227890950724E7 3718 +2.7682552632044956E7 3719 +82.46056964113751 3720 +7.825291121830407 3721 +6.103292943651967E7 3722 +2.7682552546291735E7 3723 +3.7090213413284756E7 3724 +5.114352043109575E7 3725 +9719024.174028324 3726 +9719023.49160264 3727 +2.1273499486709893E7 3728 +4.498021905882171 3729 +5.114352142818702E7 3730 +1.220273289674659E7 3731 +4.597354930924142E7 3732 +3.709021306860684E7 3733 +4.597354901907007E7 3734 +10.508444104675132 3735 +6.1032936911063E7 3736 +2.7682577660832047E7 3737 +3.1518663008366863 3738 +2.7682552146133143E7 3739 +9719023.714392055 3740 +4.5973549101697356E7 3741 +1.2202733333358089E7 3742 +4.597354940734593E7 3743 +1.2202734088741452E7 3744 +2.1273498573295604E7 3745 +6.1032932171349585E7 3746 +5.7599627465613246E7 3747 +6.103292674105978E7 3748 +5.759962712891815E7 3749 +1.2202733254015917E7 3750 +3.7090212914682485E7 3751 +2.7682551536783557E7 3752 +22.035914757270497 3753 +5.114352072417018E7 3754 +2.7682552414159443E7 3755 +4.5973549310912974E7 3756 +3.709021251047712E7 3757 +13.328605283639552 3758 +3.709021238325932E7 3759 +4.5973549169773646E7 3760 +2.7682552169156298E7 3761 +67.77549732545887 3762 +5.114352635256592E7 3763 +3.709021324549979E7 3764 +5.759962770488065E7 3765 +5.759963009703466E7 3766 +4.052351390328979 3767 +8.198572993165469 3768 +2.12753968132355E7 3769 +2.768255152548304E7 3770 +9719023.078110004 3771 +2.7682551923746996E7 3772 +1.2202733466980655E7 3773 +6.103292678135411E7 3774 +2.7682552974380173E7 3775 +2.1273505441872735E7 3776 +9719023.115174545 3777 +3.709021763092445E7 3778 +9719023.569863407 3779 +9719023.62866979 3780 +1.2202733331473531E7 3781 +6.103292747483388E7 3782 +3.2067757503896113 3783 +2.7682551879064955E7 3784 +6.103292770013216E7 3785 +5.7599627263478674E7 3786 +5.75996275014519E7 3787 +5.1143520493043914E7 3788 +3.709021549432227E7 3789 +6.10329277187878E7 3790 +6.103292737702433E7 3791 +3.709021258598479E7 3792 +6.103292785885313E7 3793 +3.7090222982935235E7 3794 +6.103292658377907E7 3795 +2.76825516970536E7 3796 +7.412693845238463 3797 +5.7599630710354105E7 3798 +2.7682555866347034E7 3799 +11.099523615332668 3800 +4.59735493324476E7 3801 +3.709021237895424E7 3802 +3.709021359051514E7 3803 +2.7682551927833818E7 3804 +1.2202733059091628E7 3805 +5.759962731800053E7 3806 +9719023.004225576 3807 +11.587828216304091 3808 +9719023.591045437 3809 +5.114352125669062E7 3810 +5.114352136182429E7 3811 +5.11435211315803E7 3812 +5.114352055788541E7 3813 +5.7599627247433536E7 3814 +9719023.034529893 3815 +1.2202732976093985E7 3816 +3.7090212679068714E7 3817 +3.709021284551919E7 3818 +3.7090212533558026E7 3819 +9719023.02042473 3820 +5.1143520673538946E7 3821 +5.114352062180773E7 3822 +9719025.955485864 3823 +1.2202733233985962E7 3824 +64.51610529942752 3825 +6.103292673147377E7 3826 +19.163234718222576 3827 +4.597354888967939E7 3828 +5.114352101157536E7 3829 +1.220273460551151E7 3830 +4.5973548877528034E7 3831 +6.1032928410779946E7 3832 +3.709021619413945E7 3833 +5.1143520392049454E7 3834 +5.759962757088086E7 3835 +4.597354891747553E7 3836 +9719023.009594427 3837 +2.768255305718043E7 3838 +6.103292670847416E7 3839 +6.103292678190823E7 3840 +5.759962721852511E7 3841 +6.103292657995218E7 3842 +4.526933007278764 3843 +2.7682552451620486E7 3844 +2.1273499732396338E7 3845 +10.971001640429996 3846 +2.7682551765456695E7 3847 +2.7683880708533842E7 3848 +2.768255247036335E7 3849 +5.7599627438356355E7 3850 +6.103292709624976E7 3851 +5.759962760444959E7 3852 +2.12734985346739E7 3853 +2.1273498336658366E7 3854 +2.12734984032313E7 3855 +2.127349901449833E7 3856 +2.1273499392423697E7 3857 +5.7599627343649924E7 3858 +5.759962768590797E7 3859 +2.768255295938488E7 3860 +3.709021305401973E7 3861 +3.7090213711479045E7 3862 +6.103292784512526E7 3863 +3.709021328404673E7 3864 +21.93588616249526 3865 +9719023.176786521 3866 +5.759962787365152E7 3867 +5.759962751216832E7 3868 +1.2202733083722645E7 3869 +4.597354919661928E7 3870 +5.759962709704732E7 3871 +3.709021487674253E7 3872 +4.597354982228024E7 3873 +9719023.536936298 3874 +5.759962710911031E7 3875 +6.103292747036504E7 3876 +2.127349835525711E7 3877 +12.182921572404398 3878 +1.22027328612943E7 3879 +1.2202733334573798E7 3880 +2.1273499636342276E7 3881 +4.597355149147523E7 3882 +5.114354071660671E7 3883 +4.5973549353786334E7 3884 +9719023.285699904 3885 +2.7682552653216034E7 3886 +6.1033301177056186E7 3887 +5.114352092372161E7 3888 +9719023.359836586 3889 +5.75996273671992E7 3890 +5.7599631274278775E7 3891 +1.2202734028062688E7 3892 +1.2202733093579847E7 3893 +1.220273604485595E7 3894 +1.220273343009359E7 3895 +2.1273498392330237E7 3896 +1.2202732907347206E7 3897 +5.7599627474390715E7 3898 +5.114352053017289E7 3899 +2.1273503416020047E7 3900 +3.709021242962996E7 3901 +1.2202733668729883E7 3902 +3.709021289081581E7 3903 +1.2202733701296262E7 3904 +15.14103983551085 3905 +9719023.309180655 3906 +2.76825516670501E7 3907 +2.7682553374709386E7 3908 +4.5973548988853596E7 3909 +2.768255271496282E7 3910 +3.709021781985125E7 3911 +3.70902121261731E7 3912 +4.597354929461716E7 3913 +2.12735393537069E7 3914 +4.597354952211791E7 3915 +1.2202751000760155E7 3916 +3.709021362069712E7 3917 +2.768255236079414E7 3918 +5.114352178198832E7 3919 +5.114352628680552E7 3920 +3.709021253831777E7 3921 +47.87660782436285 3922 +6.1032926902003795E7 3923 +2.768255210231555E7 3924 +5.7599627892040014E7 3925 +25.45185660830155 3926 +9719023.23327272 3927 +3.709021214845481E7 3928 +2.7682554334537253E7 3929 +2.7682551952048674E7 3930 +2.768255187511714E7 3931 +6.103292793328259E7 3932 +1.2202733205047557E7 3933 +6.103292724179955E7 3934 +18.30114842293245 3935 +6.103292676841486E7 3936 +5.7599627143896766E7 3937 +3.7090212417146556E7 3938 +6.103293049224758E7 3939 +7.184929741819637 3940 +9719023.218862385 3941 +1.2202733840239407E7 3942 +5.759962729478625E7 3943 +2.7682552006105985E7 3944 +2.127350026232304E7 3945 +4.11313850786355 3946 +9719023.021363119 3947 +4.5973549048103385E7 3948 +2.1273498386387646E7 3949 +5.75996286104723E7 3950 +5.114352076440558E7 3951 +4.59735494692405E7 3952 +2.7682554003081262E7 3953 +2.127349902337845E7 3954 +6.381311221502732 3955 +14.892523818009153 3956 +2.127349928699984E7 3957 +1.220273342652372E7 3958 +5.114352070416309E7 3959 +6.103292666590957E7 3960 +4.597355156473921E7 3961 +5.759962809034873E7 3962 +5.7599627324882895E7 3963 +3.709021237728778E7 3964 +3.709021265630131E7 3965 +6.103292820641674E7 3966 +2.7682556793409504E7 3967 +2.76825518677445E7 3968 +9719024.702801239 3969 +30.50307776615458 3970 +5.759962712207417E7 3971 +2.7682552225889225E7 3972 +2.7682600738025207E7 3973 +4.597354947306195E7 3974 +5.114352041454856E7 3975 +1.2202733788215272E7 3976 +2.1273509123659078E7 3977 +3.709021265979645E7 3978 +5.7599627027932525E7 3979 +5.1143520402285874E7 3980 +2.1273499096707653E7 3981 +5.75996270533456E7 3982 +3.7090212191702515E7 3983 +5.452575124054425 3984 +1.2202734541498093E7 3985 +5.7599634984266415E7 3986 +6.103292659952086E7 3987 +1.2202734265144877E7 3988 +3.709021239344962E7 3989 +9719023.468466094 3990 +5.759962731071394E7 3991 +5.7599631209137E7 3992 +2.127350025573628E7 3993 +2.1273499667904373E7 3994 +6.103292685916648E7 3995 +1.220273418477696E7 3996 +6.103292727211219E7 3997 +4.5973549569502376E7 3998 +18.891262467317866 3999 +4.597354921106498E7 4000 +4.5973548888605796E7 4001 +1.2202732863659823E7 4002 +5.759962716974142E7 4003 +2.7682552012075473E7 4004 +3.1681695650708157 4005 +9719023.415677318 4006 +2.7682552270074E7 4007 +2.7682553762150023E7 4008 +9719023.063775849 4009 +1.22027547418375E7 4010 +3.709021221590284E7 4011 +3.709021293302779E7 4012 +1.2202733012043633E7 4013 +4.5973549385732464E7 4014 +6.103292733395719E7 4015 +2.1273498690685485E7 4016 +4.597354916398821E7 4017 +3.709021552787439E7 4018 +9719023.393957883 4019 +2.7682552403211217E7 4020 +5.759962727618047E7 4021 +1.2202748994218985E7 4022 +12.996900978931968 4023 +6.103292748170856E7 4024 +12.944357142407892 4025 +23.300203623945947 4026 +3.709021210224429E7 4027 +2.7682553091715124E7 4028 +5.1143520570041604E7 4029 +5.114352073967747E7 4030 +5.114352298338516E7 4031 +4.59735492839002E7 4032 +7.419585043982673 4033 +1.2202733007328697E7 4034 +5.759962754313968E7 4035 +2.768255843633254E7 4036 +4.597354917239522E7 4037 +112.72971869477641 4038 +9719023.118419793 4039 +9719024.127508681 4040 +5.11435213837301E7 4041 +9719023.263091942 4042 +2.1273498875439063E7 4043 +4.597355226626999E7 4044 +95.62034009935451 4045 +6.10329267534173E7 4046 +9719023.587017858 4047 +5.114352420107295E7 4048 +6.103292904581664E7 4049 +2.1273499186116908E7 4050 +5.759962711253558E7 4051 +31.20739172732938 4052 +3.709021404536303E7 4053 +5.75996278443645E7 4054 +6.1032933637466736E7 4055 +3.7090212231808536E7 4056 +2.1273499036150027E7 4057 +6.103292736915199E7 4058 +3.709021256101325E7 4059 +1.2202736579635955E7 4060 +1.2202734004660567E7 4061 +5.114352063409748E7 4062 +4.597354880704933E7 4063 +2.7682556715815563E7 4064 +9719023.200868426 4065 +29.37662902258755 4066 +2.7682555611003287E7 4067 +2.7682552296471346E7 4068 +5.1143521004769064E7 4069 +6.103292744781979E7 4070 +5.759962784495609E7 4071 +5.759962868779756E7 4072 +9719023.219470773 4073 +1.2202741039887337E7 4074 +9719023.133236172 4075 +4.597354909910632E7 4076 +5.1143520547090694E7 4077 +2.127349884517711E7 4078 +9719023.134239402 4079 +5.114352208150938E7 4080 +2.7682551904165495E7 4081 +1.2202733440451553E7 4082 +9719023.291016249 4083 +1.2202733170663945E7 4084 +9719024.342843983 4085 +6.103292725526812E7 4086 +3.709021686435884E7 4087 +3.709021296527184E7 4088 +5.759962706736885E7 4089 +4.5973549801509194E7 4090 +6.1032926658327535E7 4091 +17.52000593620763 4092 +1.2202732906674968E7 4093 +3.709021260728645E7 4094 +1.220273330256177E7 4095 +5.114352105404731E7 4096 +66.55679493854309 4097 +9719023.229166858 4098 +3.709021298629398E7 4099 +2.76825520602131E7 4100 +2.7682552093062803E7 4101 +2.7682558523081195E7 4102 +2.768255205487635E7 4103 +5.1143520520915665E7 4104 +5.114352667094035E7 4105 +5.759962774449178E7 4106 +5.7599627084088005E7 4107 +5.114352108716577E7 4108 +3.709021304254568E7 4109 +104.75992172431124 4110 +5.759962708556647E7 4111 +4.597354891327719E7 4112 +2.7682552406706348E7 4113 +9719023.403347308 4114 +9719023.057623079 4115 +3.70902182792059E7 4116 +2.1273498462745313E7 4117 +2.127349870195808E7 4118 +23.11314403564997 4119 +13.515667186127652 4120 +5.114352107823332E7 4121 +4.597354938731799E7 4122 +9719023.04602203 4123 +5.759962710954861E7 4124 +9719023.18702557 4125 +5.759962727704988E7 4126 +4.597354908218951E7 4127 +6.103292707923773E7 4128 +17.25300807954829 4129 +4.5973549381633535E7 4130 +34.55890633944766 4131 +5.114352093763429E7 4132 +5.114352077758894E7 4133 +5.1143525622119665E7 4134 +5.1143522373528756E7 4135 +6.103292672677951E7 4136 +5.114352148891166E7 4137 +335.9435094127411 4138 +19.974525328804457 4139 +1.2202734273555687E7 4140 +5.114352673381052E7 4141 +5.759962771803125E7 4142 +13.166700774598672 4143 +6.1032980836844444E7 4144 +5.114352050988015E7 4145 +6.103292747284796E7 4146 +6.1032927412339136E7 4147 +5.759962756687205E7 4148 +5.114352080671009E7 4149 +4.5973549007334515E7 4150 +6.103292824009605E7 4151 +4.5973549196117856E7 4152 +9719023.353599608 4153 +6.1032932878352396E7 4154 +5.759962773479277E7 4155 +9719023.029700637 4156 +5.114352086741781E7 4157 +2.1273499979000155E7 4158 +3.709021204340149E7 4159 +5.1143520418597125E7 4160 +2.1273498335615426E7 4161 +2.127349837672828E7 4162 +2.7682552128292676E7 4163 +8.772988007415805 4164 +2.127349886775405E7 4165 +6.1032927393238015E7 4166 +3.7090212843735546E7 4167 +1.2202733774365801E7 4168 +4.597354894405546E7 4169 +2.7682551545541752E7 4170 +6.103292883045065E7 4171 +4.5973549521892734E7 4172 +35.208811845135365 4173 +9.228395367576429 4174 +6.10329274915947E7 4175 +3.709021213381089E7 4176 +19.736338572095473 4177 +2.1273498305054456E7 4178 +1.220273481307141E7 4179 +9719023.42333585 4180 +9719023.24197172 4181 +9719023.41713102 4182 +2.7682552075155098E7 4183 +9719023.608471815 4184 +9719024.362995343 4185 +4.597354930646029E7 4186 +1.2202734019297296E7 4187 +4.597354983435346E7 4188 +6.663883009064045 4189 +5.7599627102683775E7 4190 +6.103292736512545E7 4191 +3.70902155148959E7 4192 +4.597354914705493E7 4193 +5.75996278922257E7 4194 +4.5973555449050464E7 4195 +5.759962793102211E7 4196 +66.06048157929763 4197 +2.127350432738963E7 4198 +5.759962714227243E7 4199 +9719023.124408824 4200 +4.597355288161272E7 4201 +20.360899831037806 4202 +9719023.26773415 4203 +3.7090212343568444E7 4204 +2.7682552350472316E7 4205 +2.7682554444424603E7 4206 +1.2202733752440454E7 4207 +1.2202735551208204E7 4208 +4.597354947634702E7 4209 +9719023.583120666 4210 +3.7090212788125314E7 4211 +2.768255159243379E7 4212 +5.1143520614026494E7 4213 +5.759962719644037E7 4214 +3.709021300184693E7 4215 +5.114352041490528E7 4216 +7.438229927444142 4217 +6.103292789194234E7 4218 +3.709026922609274E7 4219 +6.1032936127801694E7 4220 +9719023.215827534 4221 +3.709021466915578E7 4222 +2.1273498724899173E7 4223 +2.7682636963492416E7 4224 +3.709021314906245E7 4225 +2.1273509901072193E7 4226 +5.759962703537579E7 4227 +2.7682553437441844E7 4228 +3.709021292211322E7 4229 +2.1273498966572307E7 4230 +3.709021286032524E7 4231 +6.10329348898413E7 4232 +6.1032926700254805E7 4233 +2.1273498886514504E7 4234 +2.1273499262630507E7 4235 +6.1032926971704334E7 4236 +2.7682553007758953E7 4237 +2.127350824292473E7 4238 +9719023.244912578 4239 +12.10676529437673 4240 +10.597656643710518 4241 +9719023.245825887 4242 +6.1032927271388955E7 4243 +2.1273498523510747E7 4244 +3.7090212525529325E7 4245 +41.63714705967741 4246 +9719023.491911909 4247 +2.127349913029454E7 4248 +5.1143520443231866E7 4249 +6.1032927983972654E7 4250 +5.114352095191453E7 4251 +4.5973548884813406E7 4252 +23.331083865941324 4253 +5.7599627103969276E7 4254 +1.2202734331125006E7 4255 +41.781891878113974 4256 +12.292041669833589 4257 +6.1032928443058975E7 4258 +5.1143521266150035E7 4259 +1.220273474187211E7 4260 +6.1032926790472925E7 4261 +91.87418890127894 4262 +4.59735491178415E7 4263 +6.103292665903227E7 4264 +2.127349860088291E7 4265 +5.114352074289889E7 4266 +5.759962736586054E7 4267 +9719023.526367478 4268 +9719023.20841904 4269 +6.103292664645299E7 4270 +3.7090213373604245E7 4271 +7.627666067932803 4272 +97.88744695496945 4273 +6.103292661821115E7 4274 +3.709021244551043E7 4275 +5.759962718419783E7 4276 +2.7682552366025433E7 4277 +3.709021667235463E7 4278 +5.1143527572265774E7 4279 +3.70902159466489E7 4280 +5.114352211142631E7 4281 +1.2202733669269156E7 4282 +3.709021449144211E7 4283 +5.759962704113652E7 4284 +5.759962730505698E7 4285 +3.709021256376418E7 4286 +3.709021339344958E7 4287 +9719023.160755116 4288 +6.1032927791078664E7 4289 +4.59735507871824E7 4290 +3.709021465931526E7 4291 +4.597354965423229E7 4292 +2.7682553517270274E7 4293 +5.759962720085508E7 4294 +5.11435240633073E7 4295 +5.114352076309189E7 4296 +4.597355209221867E7 4297 +2.1273498693610165E7 4298 +5.114352189010116E7 4299 +9719023.322090106 4300 +5.759962716486272E7 4301 +2.768255170347488E7 4302 +4.597356181719699E7 4303 +9719023.498317128 4304 +3.70902128228166E7 4305 +4.597354900628384E7 4306 +2.768255165830186E7 4307 +9719023.29838982 4308 +2.127349917629738E7 4309 +5.759962712815098E7 4310 +2.1273500130490843E7 4311 +1.2202737637773808E7 4312 +6.103292678278801E7 4313 +3.709021309318513E7 4314 +5.759962983748672E7 4315 +2.768255207296762E7 4316 +6.1032926564026535E7 4317 +9719023.82382051 4318 +5.114352813854767E7 4319 +11.741530020408351 4320 +3.709021864518238E7 4321 +5.1143521537245445E7 4322 +10.496798526785206 4323 +5.114352069292611E7 4324 +5.114352075645692E7 4325 +2.1273499074009556E7 4326 +2.127349977945039E7 4327 +55.34783863604856 4328 +3.709021438842347E7 4329 +4.597354907112069E7 4330 +2.7682552501214363E7 4331 +5.114352111577483E7 4332 +4.5973549552282035E7 4333 +5.114352119043061E7 4334 +4.597354925599798E7 4335 +6.103292723075834E7 4336 +2.127349895942744E7 4337 +2.7682551938395657E7 4338 +9719023.199503873 4339 +4.597354957765268E7 4340 +3.709021268367466E7 4341 +2.768255200946015E7 4342 +2.1273499032782264E7 4343 +5.114353765167864E7 4344 +9719023.30057857 4345 +4.597354926547702E7 4346 +2.1273499231715616E7 4347 +9719023.39525208 4348 +30.596683163763917 4349 +4.597354882413905E7 4350 +6.103292682717061E7 4351 +4.597354915547459E7 4352 +3.709021346914772E7 4353 +1.2202734278673382E7 4354 +6.103292773734785E7 4355 +5.114352083818207E7 4356 +3.7090212348934256E7 4357 +9719023.278425355 4358 +6.10329273809805E7 4359 +2.127349840838106E7 4360 +9719023.378372274 4361 +6.103293342627926E7 4362 +1.2202734252552731E7 4363 +5.114352050247446E7 4364 +6.103292840057965E7 4365 +2.1273498374467906E7 4366 +3.709021384405183E7 4367 +1.2202733312906109E7 4368 +6.103292706738289E7 4369 +6.103292657473709E7 4370 +5.1143521165495776E7 4371 +5.759962740419348E7 4372 +6.103292709817396E7 4373 +6.25220371321534 4374 +5.7599627596322075E7 4375 +6.103292860449184E7 4376 +2.1273498464502376E7 4377 +4.597354938483636E7 4378 +1.2202734411313113E7 4379 +6.103292671751869E7 4380 +27.137277411209173 4381 +5.11435212984347E7 4382 +2.7682552176233377E7 4383 +4.5973550364252195E7 4384 +1.220273337816933E7 4385 +4.285648074972079 4386 +5.759962724194432E7 4387 +2.7682556238009647E7 4388 +2.7682553393585086E7 4389 +2.7682553078431934E7 4390 +3.709021317161639E7 4391 +9719023.141140264 4392 +11.189250526833963 4393 +5.114352081169988E7 4394 +4.597354919725544E7 4395 +2.7682552121084176E7 4396 +2.1273499417930663E7 4397 +1.2202736298752747E7 4398 +2.768255230547213E7 4399 +1.2202733482952822E7 4400 +2.1273499264411915E7 4401 +5.1143521043698475E7 4402 +6.103292719243818E7 4403 +5.114352069327407E7 4404 +3.709021317458808E7 4405 +33.62178278332511 4406 +4.597355127892529E7 4407 +16.1881851210389 4408 +2.768255256306832E7 4409 +12.279196935222 4410 +5.759962859158854E7 4411 +5.7599627931357756E7 4412 +2.7682551754687868E7 4413 +3.709021210658959E7 4414 +5.1143522358410254E7 4415 +5.114352096947851E7 4416 +2.7682553016833767E7 4417 +7.578522722969978 4418 +4.597354989568709E7 4419 +6.103292716977718E7 4420 +1.2202733323095027E7 4421 +5.1143520974247046E7 4422 +3.709021283465808E7 4423 +2.768255543530976E7 4424 +32.06776236687575 4425 +3.709021318126939E7 4426 +5.1143522278055005E7 4427 +5.759962768370678E7 4428 +6.103292687341557E7 4429 +2.1273498681229655E7 4430 +5.114352269152077E7 4431 +2.127349937677296E7 4432 +9719023.637235409 4433 +2.7682553321055282E7 4434 +5.114352063126669E7 4435 +4.597354899459219E7 4436 +3.709021388505061E7 4437 +9719023.246039493 4438 +3.7090214661038734E7 4439 +6.10329274903735E7 4440 +6.103292658008125E7 4441 +70.91334852768026 4442 +3.709021560935637E7 4443 +6.103292738974927E7 4444 +9719023.070291495 4445 +59.341241432485376 4446 +1.2202733418759031E7 4447 +9719023.150011797 4448 +5.114352127563968E7 4449 +5.7599627321177505E7 4450 +3.709021215261056E7 4451 +2.76825519022969E7 4452 +3.709021261526835E7 4453 +5.759962958275884E7 4454 +5.114352112964807E7 4455 +1699.3858231727995 4456 +2.1273500194492754E7 4457 +5.7599627427303545E7 4458 +3.7090212743046716E7 4459 +2.12735019366275E7 4460 +3.709021292182794E7 4461 +5.114352056209432E7 4462 +9719023.037987141 4463 +2.127350077202649E7 4464 +5.7599628854708806E7 4465 +2.768255306421496E7 4466 +9719023.162169062 4467 +6.103292838966077E7 4468 +4.5973550886155605E7 4469 +4.597354935713783E7 4470 +5.7599627960754335E7 4471 +5.11435210044514E7 4472 +2.7682552530005988E7 4473 +9719023.136872744 4474 +2.768255262735422E7 4475 +3.7090214766601846E7 4476 +5.759962715745703E7 4477 +19.552522872360075 4478 +9719023.415529462 4479 +6.1032928198330276E7 4480 +5.759962755371169E7 4481 +6.103292755324703E7 4482 +9719023.002007155 4483 +5.75996272645881E7 4484 +5.114352094720559E7 4485 +3.709022037369141E7 4486 +2.768255200791098E7 4487 +5.114352110010482E7 4488 +9719023.286484396 4489 +5.759962725118859E7 4490 +3.709021422190739E7 4491 +5.7599627099699244E7 4492 +1.2202733891901325E7 4493 +2.1273499124774933E7 4494 +2.127349906256302E7 4495 +23.154473762961075 4496 +1.2202733918931374E7 4497 +160.8931054273132 4498 +5.759962764131021E7 4499 +9719023.631883023 4500 +6.103292731001462E7 4501 +2.1273499692530327E7 4502 +1.2202734646340689E7 4503 +5.114352176977111E7 4504 +5.1143521236696586E7 4505 +3.0560436286066017 4506 +4.597354910191315E7 4507 +5.1143521466061905E7 4508 +4.597354923777353E7 4509 +63.81676774890508 4510 +1.2202733937872238E7 4511 +1.2202733469502363E7 4512 +5.759962744195437E7 4513 +5.759962730346208E7 4514 +2.1273504133000545E7 4515 +8.048324086632444 4516 +3.709021262920075E7 4517 +1.2202733721409177E7 4518 +4.597354963351658E7 4519 +2.7682553032677922E7 4520 +3.70902165020379E7 4521 +5.114352681955062E7 4522 +3.70902123102882E7 4523 +6.1032930760770746E7 4524 +5.75996277227993E7 4525 +5.119442381865002 4526 +2.7682552496066723E7 4527 +2.7682551925440058E7 4528 +7.633293937381666 4529 +5.759962746629603E7 4530 +3.709022344644285E7 4531 +1.2202732942393767E7 4532 +6.103292679676205E7 4533 +3.70902170088058E7 4534 +2.7682555031689417E7 4535 +9719023.063676124 4536 +6.103292654157452E7 4537 +2.1273499692216802E7 4538 +4.5973552667304315E7 4539 +4.5973550061337896E7 4540 +3.709021567176401E7 4541 +1.2202744844447978E7 4542 +3.709021375822112E7 4543 +4.5973548891223E7 4544 +4.597354943102749E7 4545 +3.709021232609441E7 4546 +1.2202733655713229E7 4547 +6.10329266522595E7 4548 +2.1273498335435897E7 4549 +6.103294258546951E7 4550 +2.127349898472977E7 4551 +2.1273498424297664E7 4552 +5.447343902982069 4553 +3.709021294577327E7 4554 +9719023.313454239 4555 +5.114358131548817E7 4556 +6.1032926515654415E7 4557 +2.768255198205925E7 4558 +5.11435215059667E7 4559 +5.114352065497427E7 4560 +6.103293090495649E7 4561 +2.1273499223963648E7 4562 +3.709021207718837E7 4563 +3.709021298426749E7 4564 +2.7682552986364935E7 4565 +2.76825519124675E7 4566 +2.768255183815025E7 4567 +6.10329275188188E7 4568 +9719023.19098658 4569 +9719023.790754026 4570 +4.597354892147316E7 4571 +2.768255223104978E7 4572 +2.1273501357023288E7 4573 +5.759962741004685E7 4574 +2.768255285269237E7 4575 +3.709021227359635E7 4576 +1.220273745690658E7 4577 +4.59735489875442E7 4578 +4.5973549370540045E7 4579 +1.2202734951441463E7 4580 +5.759962709070835E7 4581 +9719023.223110713 4582 +1.2202733582288215E7 4583 +2.7682553017926916E7 4584 +9719023.34012472 4585 +5.1143521007501386E7 4586 +9719023.249000853 4587 +39.55234261737617 4588 +4.59735498671513E7 4589 +31.350599389362504 4590 +9719023.119032966 4591 +2.7682552296244033E7 4592 +6.103292660192277E7 4593 +2.1273498563608952E7 4594 +5.75996274739314E7 4595 +5.114352158348047E7 4596 +9719023.364533514 4597 +9719023.584637128 4598 +1.2202733876478968E7 4599 +2.1273498723680627E7 4600 +5.1143520565810554E7 4601 +5.7599627393893726E7 4602 +2.127350317197711E7 4603 +1.2202733005883694E7 4604 +4.597355126653882E7 4605 +1.2202732980409943E7 4606 +15.03725760828772 4607 +12.662212153260867 4608 +2.7682555825359363E7 4609 +6.103292755893661E7 4610 +13.101349408671272 4611 +9719025.345306285 4612 +6.103292732812305E7 4613 +2.768255560526181E7 4614 +1.220273293647394E7 4615 +4.459197035577869 4616 +2.7682551744699392E7 4617 +7.8268013112510415 4618 +2.7682551645758882E7 4619 +5.114352135025493E7 4620 +3.709021317316801E7 4621 +2.768255204691172E7 4622 +5.114352098997724E7 4623 +5.759962765669938E7 4624 +4.597354880927008E7 4625 +9719023.21325143 4626 +3.709021223581436E7 4627 +1.2202738201466028E7 4628 +9719023.072284073 4629 +5.1143520961501144E7 4630 +5.114352109298908E7 4631 +1.2202733594163463E7 4632 +9719024.376622612 4633 +5.75996277705516E7 4634 +172.31961823142436 4635 +5.256776399870903 4636 +5.114353522054965E7 4637 +2.768255381013308E7 4638 +6.103292747496179E7 4639 +5.759962760063621E7 4640 +2.1273499778680354E7 4641 +3.709022386188473E7 4642 +4.163570224729358 4643 +5.7599627077177376E7 4644 +4.597354886815595E7 4645 +10.380119592036506 4646 +2.1273498667223282E7 4647 +9719028.013906505 4648 +16.393750424822475 4649 +2.7682553152154256E7 4650 +5.114352141731241E7 4651 +3.7090253836929224E7 4652 +1.2202733690571994E7 4653 +5.7599628220546536E7 4654 +29.91609554616354 4655 +1.2202733516147207E7 4656 +1.220273307741076E7 4657 +2.127349863513877E7 4658 +4.59735490600161E7 4659 +4.597354927918407E7 4660 +14.231557674761074 4661 +9719023.276078425 4662 +2.127349850710101E7 4663 +3.709021263978359E7 4664 +2.1273498315792173E7 4665 +1.2202736588090422E7 4666 +6.1032962236156926E7 4667 +6.103292703873057E7 4668 +1.220273344442827E7 4669 +4.597360143893255E7 4670 +5.114352115728895E7 4671 +5.75996272777649E7 4672 +4.5973550723421775E7 4673 +2.768255241907623E7 4674 +5.759962723850307E7 4675 +9719023.222586693 4676 +2.1273498403320756E7 4677 +9719023.103730965 4678 +1.2202734012738714E7 4679 +2.76825519715074E7 4680 +1.2202734178334165E7 4681 +9719023.598865494 4682 +1.220273354271123E7 4683 +9719023.162536122 4684 +6.103292661505928E7 4685 +4.597354890558356E7 4686 +3.709021297329718E7 4687 +5.759962710130207E7 4688 +2.127350112463509E7 4689 +1.2202734413107304E7 4690 +3.709021232871481E7 4691 +9719023.623341365 4692 +6.103292703455619E7 4693 +9719023.03872495 4694 +5.114352176511186E7 4695 +10.940366083801596 4696 +5.7599627541826606E7 4697 +1.2202733789495986E7 4698 +5.75996273856759E7 4699 +5.759962730995045E7 4700 +1.220273388278626E7 4701 +9719023.946088988 4702 +7.886783278375633 4703 +6.10329272271375E7 4704 +2.127349898699268E7 4705 +3.8192261206341174 4706 +2.7682553917260762E7 4707 +2.7682551878978614E7 4708 +2.7682552954860926E7 4709 +6.1032927256550945E7 4710 +2.7682554680459157E7 4711 +5.1143522375959985E7 4712 +2.1273498812684793E7 4713 +27.95987676659621 4714 +1.2202733824209368E7 4715 +6.1032928853320986E7 4716 +2.7682551929109246E7 4717 +1.2202735975596527E7 4718 +6.167029163033932 4719 +4.5973549164555006E7 4720 +6.1032928776514E7 4721 +4.597354966586236E7 4722 +1.2202732870619617E7 4723 +2.127349881489915E7 4724 +3.709021259499946E7 4725 +5.75996271335364E7 4726 +3.7090213376426935E7 4727 +4.597354974336968E7 4728 +6.103293035291377E7 4729 +5.1143520898100756E7 4730 +2.1273498557064444E7 4731 +1.2202732986447657E7 4732 +1.2202736390652057E7 4733 +66.93370087732433 4734 +3.709021237431781E7 4735 +6.103292666060802E7 4736 +5.114352109748583E7 4737 +5.114352136507454E7 4738 +3.709021286852009E7 4739 +5.759963410704044E7 4740 +1.220273443009837E7 4741 +2.127349931185589E7 4742 +5.759962729528586E7 4743 +1.220273373920722E7 4744 +9719023.021383842 4745 +1.220273304319116E7 4746 +3.7090212140744515E7 4747 +2.768255158551682E7 4748 +9719024.17314668 4749 +3.709021402948225E7 4750 +2.7682552107692465E7 4751 +2.127349852991265E7 4752 +2.1273498705092523E7 4753 +5.759962737261997E7 4754 +2.1273500302114107E7 4755 +5.7599627084489994E7 4756 +2.127350787478141E7 4757 +4.597354958126111E7 4758 +5.759962765835863E7 4759 +12.521181044704544 4760 +9719023.216127818 4761 +6.103292772477327E7 4762 +1.2202738680590225E7 4763 +1.2202733143385954E7 4764 +2.1273502163915277E7 4765 +4.365157571748859 4766 +4.597355881760928E7 4767 +6.1032928225053E7 4768 +41.47505506268888 4769 +9719023.038371533 4770 +6.103292770578532E7 4771 +2.7682552271690384E7 4772 +9719023.126188174 4773 +2.1273501763778914E7 4774 +3.574325942160839 4775 +18.70754349304893 4776 +6.1032926512756854E7 4777 +6.103292740593112E7 4778 +5.1143520891292796E7 4779 +2.1273498482366085E7 4780 +6.103292694718731E7 4781 +2.1273498571137175E7 4782 +4.597355362255964E7 4783 +3.709021487706405E7 4784 +3.709022047321327E7 4785 +8.439431281525337 4786 +6.103292694411576E7 4787 +3.809420183712571 4788 +3.709021331287654E7 4789 +12.803537239741267 4790 +4.597354978012704E7 4791 +5.7599627341194846E7 4792 +4.5973549397360824E7 4793 +3.709021260811082E7 4794 +6.103292685321295E7 4795 +5.114352062198811E7 4796 +5.759962740869764E7 4797 +4.597354967461193E7 4798 +1.2202735589079408E7 4799 +5.759962720797621E7 4800 +2.768255265030311E7 4801 +6.1032927069508865E7 4802 +1.2202732874554358E7 4803 +5.114352039758419E7 4804 +3.709021250695234E7 4805 +6.1032929502139844E7 4806 +5.114352108446977E7 4807 +9719023.459359763 4808 +6.1032927857852794E7 4809 +2.768255261672259E7 4810 +2.7682552950534496E7 4811 +5.7599627236974485E7 4812 +4.5973550286876716E7 4813 +2.1273498689000353E7 4814 +5.1143523077607185E7 4815 +2.1273498663585898E7 4816 +5.114352465820712E7 4817 +2.7682553218089722E7 4818 +5.114353163976955E7 4819 +4.103834509580572 4820 +5.114352095221077E7 4821 +2.1273498543190576E7 4822 +6.103292701847109E7 4823 +12.18466706347122 4824 +4.597355087428333E7 4825 +3.7090213207494706E7 4826 +1.2202734569376182E7 4827 +5.114352170016626E7 4828 +1.2202733600381808E7 4829 +5.7599627027694955E7 4830 +13.766877146932803 4831 +5.114352087781712E7 4832 +6.103292944608055E7 4833 +3.709029402296579E7 4834 +2.7682552220779438E7 4835 +2.7682552772720166E7 4836 +1.2202733550519047E7 4837 +6.103293300584668E7 4838 +1.2202734652248045E7 4839 +5.7599628614639215E7 4840 +3.709021542249497E7 4841 +1.2202733073237058E7 4842 +9719023.16934241 4843 +2.768255195956E7 4844 +1.2202739140428346E7 4845 +6.1032945375411585E7 4846 +6.103292728351338E7 4847 +3.7090214466375954E7 4848 +2.768255371120327E7 4849 +4.597354898143924E7 4850 +10.531980836816844 4851 +5.114352050728734E7 4852 +6.1032937587559335E7 4853 +6.103292840860332E7 4854 +50.4670719847459 4855 +4.5973552341341734E7 4856 +6.103292700115072E7 4857 +6.1032929708616175E7 4858 +4.597354937383589E7 4859 +5.114352074507811E7 4860 +6.10329279068731E7 4861 +6.103292658778352E7 4862 +2.1273746685800325E7 4863 +4.597354920289229E7 4864 +1.2202737181312306E7 4865 +4.597354962167241E7 4866 +4.597354883118201E7 4867 +1.2202732939901752E7 4868 +5.114352058205362E7 4869 +2.1273498530334506E7 4870 +5.026592515713375 4871 +2.768255159318583E7 4872 +3.709021209523304E7 4873 +4.7011684064487 4874 +3.709021229070686E7 4875 +4.597354887194575E7 4876 +1.220273351642905E7 4877 +1.2202733737162655E7 4878 +2.7682551904502433E7 4879 +5.7599627438945815E7 4880 +5.114352057744432E7 4881 +5.114352472762103E7 4882 +5.759962727212081E7 4883 +3.709021295883856E7 4884 +2.1273499574576464E7 4885 +6.1032927157548055E7 4886 +1.2202736601123992E7 4887 +9719023.209251242 4888 +5.759962750221783E7 4889 +5.114352094330582E7 4890 +3.709021301535134E7 4891 +1.2202733066086642E7 4892 +2.7682553186248858E7 4893 +4.597354922434865E7 4894 +2.768255153115242E7 4895 +2.768255231146975E7 4896 +6.1032927495988E7 4897 +3.7090213901353575E7 4898 +4.597354930684228E7 4899 +6.103292748319959E7 4900 +5.11435209530323E7 4901 +5.759962767424299E7 4902 +4.5973548937227756E7 4903 +5.7599627722506434E7 4904 +1.2202735185761994E7 4905 +33.609952135467104 4906 +4.597354937620789E7 4907 +4.597354996187392E7 4908 +5.114352061044549E7 4909 +2.1273498564617768E7 4910 +2.7682551800862115E7 4911 +1.2202735479382465E7 4912 +30.00494097964575 4913 +4.5973549235891014E7 4914 +2.7682558549892362E7 4915 +1.220273450496078E7 4916 +5.114352107654099E7 4917 +4.597355142379723E7 4918 +1.220273651272841E7 4919 +9719023.164765045 4920 +6.103293228239E7 4921 +1.2202733392029978E7 4922 +4.597354974201165E7 4923 +4.5973548837319225E7 4924 +4.597354925065774E7 4925 +5.114352130565224E7 4926 +9.454439481799678 4927 +4.597354913702514E7 4928 +2.7682552902099546E7 4929 +8.039727063856308 4930 +2.768255299456552E7 4931 +2.768255270118789E7 4932 +3.709021368002285E7 4933 +8.378636315659493 4934 +4.5973549036402665E7 4935 +9719023.10684343 4936 +1.2202733524712007E7 4937 +6.127281920308822 4938 +5.7599627115916155E7 4939 +3.709021420131176E7 4940 +3.709021319750969E7 4941 +3.709021341608594E7 4942 +5.114352041701842E7 4943 +20.246399052208037 4944 +2.1273499831212036E7 4945 +2.12734988670038E7 4946 +5.1143521056708045E7 4947 +3.7090222184835024E7 4948 +2.1273501031975597E7 4949 +15.13347269847241 4950 +18.976264588593256 4951 +5.114352096755301E7 4952 +3.7090212934343144E7 4953 +5.759962829864812E7 4954 +9719023.063113034 4955 +1.2202733341009447E7 4956 +6.103292775908333E7 4957 +6.865164678911881 4958 +1.2202740457261922E7 4959 +9719032.972686436 4960 +2.1273498498490125E7 4961 +4.597354951868016E7 4962 +6.1032927084785886E7 4963 +9719023.0403127 4964 +6.1032927202131756E7 4965 +9719023.10765521 4966 +5.75996274425075E7 4967 +70.28759076133261 4968 +4.59735594598833E7 4969 +3.709021424431576E7 4970 +5.759962877690613E7 4971 +5.114352163579613E7 4972 +3.709021301594069E7 4973 +1.2202733182378149E7 4974 +6.103292801553217E7 4975 +2.127349897284867E7 4976 +4.597355071467318E7 4977 +5.759962728008363E7 4978 +5.114352130136191E7 4979 +2.127349881437659E7 4980 +31.98762598544588 4981 +5.114352049262651E7 4982 +2.768255343907029E7 4983 +6.103293035807134E7 4984 +1.2202732935481446E7 4985 +3.709021409807345E7 4986 +3.709021315042705E7 4987 +5.7599627536649376E7 4988 +4.597354910417075E7 4989 +5.114352105983883E7 4990 +1.220273411993152E7 4991 +4.597354910210073E7 4992 +5.114352042664657E7 4993 +3.709021274594165E7 4994 +5.759962781874542E7 4995 +9719024.16291342 4996 +3.709021368842227E7 4997 +6.103292958482163E7 4998 +2.127349886093624E7 4999 +5.7599627267418645E7 5000 +5.1143520561788894E7 5001 +2.7682551975574378E7 5002 +4.597354913296208E7 5003 +5.759962718629077E7 5004 +6.103292697296385E7 5005 +5.1143520398519084E7 5006 +5.75996272180516E7 5007 +9719023.604102973 5008 +2.768255166250245E7 5009 +9719024.084900374 5010 +2.7682552501166835E7 5011 +4.597355037416569E7 5012 +3.7090212948156945E7 5013 +49.4972816746625 5014 +5.7599627911721855E7 5015 +5.759962746942464E7 5016 +19.890143688423095 5017 +3.709021310170546E7 5018 +6.103292655273261E7 5019 +2.1273499225373883E7 5020 +6.103292692686689E7 5021 +3.709021221187025E7 5022 +3.709021266776979E7 5023 +2.7682552283015583E7 5024 +9719024.100146603 5025 +2.1273498630392537E7 5026 +1.2202734140215155E7 5027 +8.757323231575132 5028 +3.70902126435632E7 5029 +2.7682552197634365E7 5030 +1.220273350322316E7 5031 +6.103292722079644E7 5032 +4.59735493107742E7 5033 +6.1032929517872155E7 5034 +35.43379339705155 5035 +5.1144007373787895E7 5036 +2.1273508313768946E7 5037 +4.5973548811346434E7 5038 +16.8023883557311 5039 +2.1273504902597383E7 5040 +5.759962732214961E7 5041 +9719023.248592602 5042 +9719023.001646897 5043 +2.1273498575197034E7 5044 +9719023.029450752 5045 +3.709021243889481E7 5046 +6.1032926867299E7 5047 +1.2202745402840227E7 5048 +4.839112285657507 5049 +4.5973548954291284E7 5050 +2.1273498832957167E7 5051 +4.597355017971855E7 5052 +3.709022808007988E7 5053 +1.2202733028790541E7 5054 +3.709031359667551E7 5055 +5.114352128483143E7 5056 +9719023.000801304 5057 +6.106992548009577 5058 +76.13373799768878 5059 +9719023.07501057 5060 +1.2202733059762787E7 5061 +88.97992022626482 5062 +5.114352039726268E7 5063 +6.1032926868630424E7 5064 +3.7090213879410364E7 5065 +6.103292684261283E7 5066 +4.597354920312656E7 5067 +6.103292868792787E7 5068 +3.709021249470605E7 5069 +3.709021286411935E7 5070 +5.1143520723473765E7 5071 +6.103292652337249E7 5072 +3.7090212639536954E7 5073 +2.7682552438128654E7 5074 +1.2202734122782163E7 5075 +5.75996274254817E7 5076 +3.9032935322467344 5077 +5.114352078385109E7 5078 +2.127349970382909E7 5079 +9719023.26722316 5080 +2.1273504150622994E7 5081 +6.103292684917498E7 5082 +6.103292713056934E7 5083 +4.597354983300902E7 5084 +9719023.018315466 5085 +2.1273498758432597E7 5086 +2.76825518122462E7 5087 +5.1143520899775326E7 5088 +5.12040894705946 5089 +9719023.406724256 5090 +5.759962764945134E7 5091 +5.7599627484211795E7 5092 +4.597355019077338E7 5093 +2.7682553066328675E7 5094 +4.824540481477336 5095 +12.23019086366569 5096 +6.1033042740256086E7 5097 +2.7682552914424498E7 5098 +3.709021241506725E7 5099 +3.709021398118837E7 5100 +5.114352109910969E7 5101 +11.616528935933182 5102 +3.709021399355442E7 5103 +2.127349887343824E7 5104 +2.1273499569237784E7 5105 +3.8448377684248056 5106 +2.1273499870702185E7 5107 +1.220273304264856E7 5108 +5.1143521094454244E7 5109 +1.2202733712322436E7 5110 +6.103292798605874E7 5111 +2.127350973927848E7 5112 +4.597354907507808E7 5113 +4.597354908310958E7 5114 +2.1273499254460514E7 5115 +11.199976543314168 5116 +9719023.09217019 5117 +2.1273498562983282E7 5118 +2.768255179431698E7 5119 +2.768255220271182E7 5120 +4.597354961698138E7 5121 +5.759962752988663E7 5122 +3.709022408203375E7 5123 +4.5973549124225944E7 5124 +2.768255207108439E7 5125 +4.597354978868392E7 5126 +3.709021522212754E7 5127 +3.709021254781379E7 5128 +1.2202735738264037E7 5129 +3.7090212120683186E7 5130 +9719023.189085668 5131 +5.7599627774872E7 5132 +2.7682551857814252E7 5133 +9719023.132006466 5134 +9719023.62665008 5135 +3.709021429734804E7 5136 +3.09078943337225 5137 +1.2202733795696253E7 5138 +17.025676516093387 5139 +5.759962710815121E7 5140 +2.127350039522075E7 5141 +3.709021258905462E7 5142 +5.666662265254611 5143 +5.114352093490624E7 5144 +6.103292724269649E7 5145 +2.1273498841870956E7 5146 +6.1032928804884225E7 5147 +4.597354927248678E7 5148 +6.103292706828984E7 5149 +1.2202733558763782E7 5150 +2.7682552986093756E7 5151 +6.191659999757308 5152 +6.103292693119341E7 5153 +5.759962744783666E7 5154 +6.103293037474628E7 5155 +3.709021272905689E7 5156 +2.1273499174259596E7 5157 +9719023.161264861 5158 +3.709021217371879E7 5159 +5.7599627192261055E7 5160 +1.220273314684895E7 5161 +7.217359924583402 5162 +4.597354906324232E7 5163 +6.103292723148532E7 5164 +4.597354935063771E7 5165 +1.2202733596596245E7 5166 +4.5973549051737785E7 5167 +1.2202733984194363E7 5168 +2.1273498819021903E7 5169 +4.5973548916061595E7 5170 +6.83922138911921 5171 +1.2202733654085519E7 5172 +6.103292729265474E7 5173 +2.127349846201337E7 5174 +3.709021379275315E7 5175 +5.7599627632407576E7 5176 +2.768255178984555E7 5177 +5.759962714089247E7 5178 +4.5973549415828854E7 5179 +2.7682552365581993E7 5180 +9719023.234527722 5181 +5.759962722647865E7 5182 +5.1143521302399516E7 5183 +2.127349864176883E7 5184 +3.70902138375049E7 5185 +5.114352083060483E7 5186 +2.7682555736122385E7 5187 +1.220273380820926E7 5188 +3.709021254745572E7 5189 +2.76825530119499E7 5190 +2.1273498472929243E7 5191 +4.597354907682922E7 5192 +4.5973550214070596E7 5193 +7.753231649484642 5194 +4.597355010623121E7 5195 +5.114352130912607E7 5196 +23.648691894321583 5197 +2.1273500670777924E7 5198 +9719023.32705373 5199 +6.10329276939281E7 5200 +5.114352103022353E7 5201 +9719023.539750723 5202 +4.597354888115234E7 5203 +9719023.247100947 5204 +5.759963007780508E7 5205 +2.1273498790440474E7 5206 +5.114352098658828E7 5207 +5.114352065817937E7 5208 +4.59735512573466E7 5209 +6.103292735294944E7 5210 +2.1273499635448944E7 5211 +1.2202733509533253E7 5212 +2.7682552030566037E7 5213 +2.127349898932512E7 5214 +2.127349909481759E7 5215 +9719023.037395375 5216 +2.127349860666796E7 5217 +5.759962771853591E7 5218 +5.759962722826293E7 5219 +2.1273499694434837E7 5220 +1.2202734230655186E7 5221 +5.1143520623629525E7 5222 +2.127350606953121E7 5223 +1.220273340772207E7 5224 +2.1273499598660074E7 5225 +8.183282972453496 5226 +4.597355001083766E7 5227 +5.759962747704098E7 5228 +4.597355015848941E7 5229 +1.2202734992707577E7 5230 +4.597355595003242E7 5231 +5.114352037523627E7 5232 +5.759962705708853E7 5233 +6.1032929533301584E7 5234 +9719023.095062012 5235 +3.7090213849684075E7 5236 +5.114352495286546E7 5237 +2.1273498440464836E7 5238 +5.759962871087972E7 5239 +1.2202733110210946E7 5240 +2.127349897887351E7 5241 +9719023.165314605 5242 +4.597354981238765E7 5243 +37.071829939075194 5244 +5.114352082103251E7 5245 +6.103292706572031E7 5246 +11.171091690890329 5247 +5.75996270866143E7 5248 +5.759962904373386E7 5249 +6.103292703785949E7 5250 +9719025.54251703 5251 +4.5973549410881914E7 5252 +2.768255345688595E7 5253 +3.709021267031886E7 5254 +2.7682551812984478E7 5255 +1.2202733320603777E7 5256 +4.597354951618758E7 5257 +5.7599629007764086E7 5258 +2.1273498904603943E7 5259 +4.597354895716343E7 5260 +1.2202733887073416E7 5261 +6.10329397527925E7 5262 +5.430796979323928 5263 +2.127349862314331E7 5264 +6.103292864514454E7 5265 +5.759963235291793E7 5266 +9719023.15187329 5267 +1.2202733332810283E7 5268 +2.1273498910495445E7 5269 +4.597355367233276E7 5270 +2.1273503905437898E7 5271 +2.768255371418401E7 5272 +6.103292800673718E7 5273 +9719023.898626583 5274 +9719024.98350098 5275 +9719023.06625587 5276 +1.2202733037737574E7 5277 +2.768256657951484E7 5278 +5.114352099489169E7 5279 +6.103292711507135E7 5280 +5.759962778472175E7 5281 +12.661562268563532 5282 +3.709021641828735E7 5283 +3.709021262075624E7 5284 +5.1143522659006506E7 5285 +1.2202741106395025E7 5286 +3.709021327730812E7 5287 +5.1143521060497016E7 5288 +5.114352098031815E7 5289 +4.597354883003177E7 5290 +6.1032926984059535E7 5291 +2.127350052581927E7 5292 +3.709021240163973E7 5293 +6.188587525662167 5294 +4.597354900398287E7 5295 +4.597355552717221E7 5296 +4.59735494385895E7 5297 +2.127349954205545E7 5298 +5.759962727756426E7 5299 +5.7599627232011065E7 5300 +5.7599627383437976E7 5301 +5.7599627090288885E7 5302 +4.597354979480383E7 5303 +2.127349977433888E7 5304 +5.75996274352926E7 5305 +1.2202733714420332E7 5306 +2.1273499082279775E7 5307 +2.127349852396116E7 5308 +3.709021295129499E7 5309 +4.5973550119908035E7 5310 +3.7090212168777496E7 5311 +9719023.056957003 5312 +2.7682551720718075E7 5313 +20.17717827227833 5314 +5.114352168294266E7 5315 +6.103292657030977E7 5316 +2.1273498405834693E7 5317 +9719023.518520614 5318 +2.7682552749642715E7 5319 +20.520447651828352 5320 +5.114352116481886E7 5321 +9719024.002193429 5322 +2.1273595802591935E7 5323 +60.69216228945168 5324 +3.709021411875741E7 5325 +5.75996270828558E7 5326 +3.146266596079122 5327 +8.125221407440415 5328 +3.709021280316112E7 5329 +3.709021321241334E7 5330 +6.103292701198119E7 5331 +9719024.291075302 5332 +5.114353237916897E7 5333 +3.709021439856961E7 5334 +5.649098795648965 5335 +3.709021666909657E7 5336 +6.103293908136956E7 5337 +6.1032927665731154E7 5338 +3.7090212897156075E7 5339 +9719025.487688685 5340 +5.1143521058760785E7 5341 +2.1273500280095156E7 5342 +6.103292739054558E7 5343 +6.1032926702666305E7 5344 +47.420538737776596 5345 +1.2202733179410715E7 5346 +5.114352420509135E7 5347 +2.7682552449217904E7 5348 +5.7599627358738706E7 5349 +2.1273498435114574E7 5350 +5.114352149614701E7 5351 +2.12734999732725E7 5352 +3.709021236895938E7 5353 +9719023.190026574 5354 +1.2202733854409207E7 5355 +5.1143521435058266E7 5356 +2.1273500049320336E7 5357 +6.103293268893835E7 5358 +9719023.149207316 5359 +5.1143520980780974E7 5360 +5.7599627655128144E7 5361 +2.1273498665784966E7 5362 +40.02140420636013 5363 +9719023.170564275 5364 +5.759962756413073E7 5365 +3.709022935944255E7 5366 +4.597354940553183E7 5367 +1.2202781525831994E7 5368 +2.1273499129075192E7 5369 +4.597354894593776E7 5370 +1.2202733306503393E7 5371 +3.709021322698722E7 5372 +3.7090212832522616E7 5373 +2.7682552491020333E7 5374 +3.709021266964237E7 5375 +17.27413120415152 5376 +3.7090213738648266E7 5377 +9719023.660827536 5378 +6.103292720121528E7 5379 +2.768255280153091E7 5380 +2.768255527135076E7 5381 +8.41608173607958 5382 +1.2202734383547882E7 5383 +4.597354899857724E7 5384 +9719023.840257358 5385 +2.1273498723337393E7 5386 +5.759962741793422E7 5387 +5.759962757913661E7 5388 +5.7599628254722975E7 5389 +2.768255212607475E7 5390 +6.103292688478443E7 5391 +5.7599627044712625E7 5392 +3.709021325194841E7 5393 +9719023.217722263 5394 +2.7682552169533916E7 5395 +5.75996274829732E7 5396 +2.768256571969049E7 5397 +2.7682552844074503E7 5398 +2.127349961511727E7 5399 +4.504161611237136 5400 +5.759962704931347E7 5401 +1.220273368987628E7 5402 +1.220273324755711E7 5403 +5.114352126328654E7 5404 +5.114352091072551E7 5405 +9719023.202938981 5406 +4.597354903302654E7 5407 +9.571010370239575 5408 +6.103292865533467E7 5409 +9719023.696978642 5410 +9719023.202014906 5411 +3.70902126978078E7 5412 +5.7599627373407975E7 5413 +3.70902125651611E7 5414 +4.900435432141403 5415 +9719023.210077597 5416 +9719023.279501317 5417 +5.114352043355986E7 5418 +5.7599631806557044E7 5419 +6.103292777222276E7 5420 +2.768255178067616E7 5421 +6.103292698635134E7 5422 +5.7599627337320335E7 5423 +3.70902130371138E7 5424 +6.10329360851637E7 5425 +3.709021210888517E7 5426 +6.1032928402206436E7 5427 +7.225981269995153 5428 +11.856636990744303 5429 +4.597354946560217E7 5430 +1.2202733409398176E7 5431 +9719023.166835243 5432 +5.7599627644041665E7 5433 +5.759962894796162E7 5434 +1.2202748747570815E7 5435 +1.2202737354410332E7 5436 +1.2202733041381534E7 5437 +3.709021232244489E7 5438 +4.597354950097162E7 5439 +1.220273287015885E7 5440 +5.488323658261844 5441 +12.409944993659984 5442 +6.690474863090771 5443 +16.892486324742904 5444 +4.5973549019993424E7 5445 +9719023.082195569 5446 +5.7599627400657594E7 5447 +4.597354958795033E7 5448 +5.75996275244562E7 5449 +5.11435209701435E7 5450 +2.1273498797703445E7 5451 +3.7090212452123426E7 5452 +3.709021253282016E7 5453 +3.70902121833983E7 5454 +5.759962769669929E7 5455 +1.220273561344236E7 5456 +6.103293006987145E7 5457 +5.7599627619005054E7 5458 +5.114352082637949E7 5459 +5.7599627037219115E7 5460 +5.114352079854967E7 5461 +9719023.582357157 5462 +5.114352073689939E7 5463 +2.127349844734266E7 5464 +5.759962722271675E7 5465 +11.115947081723819 5466 +1.2202733259233123E7 5467 +6.103292877142926E7 5468 +5.759962720312827E7 5469 +2.7682551888950802E7 5470 +5.114352200445157E7 5471 +5.75996272575683E7 5472 +2.1273498836838625E7 5473 +9719023.082229774 5474 +2.127349845465054E7 5475 +2.7682553255195115E7 5476 +9719023.53608908 5477 +6.103292779586478E7 5478 +2.1273499165422007E7 5479 +2.127349866463519E7 5480 +11.439359250775935 5481 +3.709021271381957E7 5482 +4.597355069287626E7 5483 +15.17148683391962 5484 +3.709021303030346E7 5485 +5.759962890225027E7 5486 +5.114352984809617E7 5487 +2.7682554425508007E7 5488 +3.7090213412745275E7 5489 +9719023.906660823 5490 +5.114352080222709E7 5491 +2.127349881046485E7 5492 +5.759962720145595E7 5493 +2.1273499178519238E7 5494 +1.2202733312689561E7 5495 +3.7090212637515016E7 5496 +3.709022200883284E7 5497 +5.759962777044563E7 5498 +4.5973548987208255E7 5499 +2.1273498472922225E7 5500 +2.768255296428408E7 5501 +6.1032928445556395E7 5502 +9719023.399909394 5503 +5.1143520945483826E7 5504 +2.1273498937688276E7 5505 +2.127349846114368E7 5506 +3.709021267041865E7 5507 +9719023.22356903 5508 +4.597354926736968E7 5509 +1.2202737107210128E7 5510 +5.7599627518191636E7 5511 +21.75467308669825 5512 +33.126879425234996 5513 +4.597354912823399E7 5514 +1.2202735534904312E7 5515 +9719023.036303427 5516 +16.149115237846715 5517 +5.114352121243107E7 5518 +5.759962724800016E7 5519 +6.103292740002432E7 5520 +5.7599628237439565E7 5521 +1.2202733950360818E7 5522 +4.597354934864357E7 5523 +1.2202734066644276E7 5524 +3.709021483357606E7 5525 +9719023.347653143 5526 +3.709021258396651E7 5527 +5.114352159886508E7 5528 +5.7599629930081174E7 5529 +9719023.134745175 5530 +3.709021268349853E7 5531 +5.114352156069468E7 5532 +6.1033249895531565E7 5533 +9719023.381365497 5534 +9719023.039209504 5535 +1.2202742307586852E7 5536 +12.013238239834198 5537 +6.103292795497572E7 5538 +24.152461761718847 5539 +5.75996271699815E7 5540 +4.597355263334017E7 5541 +2.768255301204772E7 5542 +6.103292739811389E7 5543 +5.145768678284914 5544 +1.2202734350258846E7 5545 +9719023.445393056 5546 +2.768255279435713E7 5547 +7.171228256553241 5548 +21.905786546757007 5549 +2.7682552562065184E7 5550 +6.103292702176301E7 5551 +2.1273498298933096E7 5552 +2.7682551529208075E7 5553 +9719023.201995548 5554 +1.220273301775161E7 5555 +9719023.185019251 5556 +4.597354938769524E7 5557 +2.1273498625879724E7 5558 +3.7090212483664565E7 5559 +1.2202736040470896E7 5560 +5.114352070430795E7 5561 +6.261937728961247 5562 +5.1143521040167585E7 5563 +5.759962713627259E7 5564 +1.220273338061017E7 5565 +26.468202152068145 5566 +3.70902124108518E7 5567 +6.103295328359627E7 5568 +9719023.49928662 5569 +1.2202733184135217E7 5570 +2.1273499687333934E7 5571 +2.1273498600546602E7 5572 +6.103292763938427E7 5573 +3.709021300095447E7 5574 +5.1143521114384025E7 5575 +4.597354906836182E7 5576 +4.5973549167388156E7 5577 +2.7682551998858493E7 5578 +1.220273393538534E7 5579 +9719023.301924007 5580 +3.709021289853456E7 5581 +1.2202734084169872E7 5582 +5.75996317086708E7 5583 +15.584734657654948 5584 +6.103293638751542E7 5585 +2.127349867448503E7 5586 +31.460685328992053 5587 +27.85171087825016 5588 +6.10329269012688E7 5589 +2.127350468583723E7 5590 +9719024.581063312 5591 +3.709021298639859E7 5592 +9719023.358835688 5593 +6.103292661381433E7 5594 +3.709021290906055E7 5595 +4.651339778011821 5596 +2.7682553666673694E7 5597 +5.1143521074096784E7 5598 +5.7599627179151624E7 5599 +5.7599628397069275E7 5600 +4.597354910260086E7 5601 +13.61819891222875 5602 +1.2202733203384599E7 5603 +20.023365323627104 5604 +29.731733797893234 5605 +6.103292828595938E7 5606 +2.7682577697555E7 5607 +5.759962721315606E7 5608 +5.11435217030074E7 5609 +5.1143520629849054E7 5610 +2.7682551578277666E7 5611 +4.597354890432137E7 5612 +2.127349902318464E7 5613 +2.768255554586626E7 5614 +31.837284535865393 5615 +5.759964180439062E7 5616 +6.103292885543571E7 5617 +33.840415132862994 5618 +6.1032937330024704E7 5619 +17.182493654114815 5620 +2.127349857633165E7 5621 +1.2202732962009782E7 5622 +4.597354997778551E7 5623 +3.040398014163641 5624 +2.127349900903147E7 5625 +605.7156438504396 5626 +5.759962801691264E7 5627 +2.7682551620032948E7 5628 +6.103293082528398E7 5629 +2.1273498436390888E7 5630 +9719023.044032214 5631 +2.1273498516450487E7 5632 +1.2202733709278986E7 5633 +2.7682552362034876E7 5634 +5.759962929343587E7 5635 +6.103292722632913E7 5636 +9719023.195700645 5637 +2.1273499568965588E7 5638 +4.597354895883731E7 5639 +1.2203443746406283E7 5640 +3.7090212142287664E7 5641 +5.759962753168379E7 5642 +2.127349861236078E7 5643 +2.7682551532899022E7 5644 +2.768255174641249E7 5645 +2.1273498553623132E7 5646 +2.127349862655294E7 5647 +9719023.226617627 5648 +5.11435211660623E7 5649 +4.597354937437283E7 5650 +22.475764041856518 5651 +9719023.102779178 5652 +5.759962873674631E7 5653 +5.7599628141252704E7 5654 +4.597354890687672E7 5655 +9719023.033308728 5656 +1.2202733294475226E7 5657 +2.1273498567361973E7 5658 +1.220273717834512E7 5659 +3.70902190784196E7 5660 +9719025.53802434 5661 +1.22027332568196E7 5662 +2.1273498635334425E7 5663 +2.7682555957180668E7 5664 +5.114353016637499E7 5665 +6.1032956607359104E7 5666 +4.597354912698511E7 5667 +9719023.033578454 5668 +1.2202733459868815E7 5669 +3.7090212887955636E7 5670 +2.7682554645946074E7 5671 +5.759962722198948E7 5672 +2.127349880865205E7 5673 +1.2202733252570806E7 5674 +6.103293348992081E7 5675 +1.2202733254099725E7 5676 +2.1273498589063738E7 5677 +5.759962888539779E7 5678 +9.28668517058083 5679 +1.2202733594711013E7 5680 +3.7090213424066894E7 5681 +2.7682552368011173E7 5682 +5.304212345933908 5683 +4.027037275108878 5684 +1.2202733121151576E7 5685 +5.6321722564552745 5686 +4.597355064320899E7 5687 +4.597354891183856E7 5688 +2.127349886994706E7 5689 +3.709021215089459E7 5690 +5.1143524585326955E7 5691 +3.709021478137732E7 5692 +2.127349989749974E7 5693 +9719023.516721351 5694 +9719055.613951892 5695 +41.48678379656054 5696 +2.7682551639625084E7 5697 +2.1273498574271895E7 5698 +2.1273498885208078E7 5699 +5.1143520852656394E7 5700 +11.163097012401995 5701 +2.7682552462252084E7 5702 +5.1143520945484996E7 5703 +1.2202734043862838E7 5704 +1.2202733966885148E7 5705 +2.1273498829931784E7 5706 +1.2202748263422009E7 5707 +6.103292865220168E7 5708 +4.5973550313324146E7 5709 +5.114352513149588E7 5710 +3.70902127274036E7 5711 +3.709021224488501E7 5712 +52.78503784248089 5713 +9719023.022615017 5714 +8.874557783084034 5715 +4.101177884652924 5716 +1.2202733600937353E7 5717 +2.7682551649122838E7 5718 +5.759962758415811E7 5719 +2.1273515290292315E7 5720 +1.2202733098054035E7 5721 +5.114352360483383E7 5722 +6.1032927107731305E7 5723 +4.597354938331788E7 5724 +6.1032927379375055E7 5725 +6.1032945862004034E7 5726 +9719023.012137473 5727 +5.1143523941854544E7 5728 +2.1273498626945443E7 5729 +4.597354921131736E7 5730 +3.7090212219571605E7 5731 +5.114352133735635E7 5732 +158.00941366664898 5733 +3.709021330859848E7 5734 +2.1273500221026327E7 5735 +2.12734985728187E7 5736 +4.175281411958232 5737 +2.7682554482383914E7 5738 +9719023.088163495 5739 +9719023.010601135 5740 +2.1273498678271126E7 5741 +4.597354967375921E7 5742 +1.2202733747832479E7 5743 +4.5973549778269306E7 5744 +1.2202736507285684E7 5745 +9719023.187723603 5746 +2.7682575481505375E7 5747 +2.768255398267268E7 5748 +3.709021406539688E7 5749 +2.1273504294558756E7 5750 +5.759963450216143E7 5751 +9719023.360692805 5752 +5.878451567062854 5753 +1.22027329481804E7 5754 +2.1273498851136148E7 5755 +4.5973549192325644E7 5756 +9719023.094757726 5757 +5.759962747863769E7 5758 +2.1273501347946394E7 5759 +9719023.194754155 5760 +3.7090212340468176E7 5761 +3.709021235688775E7 5762 +5.114352042122665E7 5763 +4.597354880143732E7 5764 +5.114352114146516E7 5765 +3.7090212171322525E7 5766 +6.103292660512726E7 5767 +5.114352120319042E7 5768 +9719023.222568277 5769 +1.2202742453364896E7 5770 +9719024.398934152 5771 +2.7682551904760323E7 5772 +5.114352052884031E7 5773 +5.759962710071331E7 5774 +2.76825562267728E7 5775 +2.7682552384946577E7 5776 +6.10329274427118E7 5777 +5.7599628284488104E7 5778 +2.127350176057966E7 5779 +4.597354954014639E7 5780 +9719023.34435997 5781 +2.127350352076041E7 5782 +4.812617162836449 5783 +1.2202733223942928E7 5784 +2.127350148178729E7 5785 +6.103292726011781E7 5786 +6.103292679457034E7 5787 +9719023.096466646 5788 +6.103292705425672E7 5789 +5.759962706278644E7 5790 +8.900371351576766 5791 +5.7599627052843735E7 5792 +2.1273500190246813E7 5793 +9719023.29577202 5794 +6.433763904931084 5795 +5.114352050031385E7 5796 +5.114352045048739E7 5797 +2.7682551729802158E7 5798 +6.1032926848726094E7 5799 +4.59735538063473E7 5800 +5.759963055227296E7 5801 +3.70902121957093E7 5802 +9719023.35017581 5803 +2.1273498732727986E7 5804 +5.114352079088667E7 5805 +67.5190184273857 5806 +6.1032927873501025E7 5807 +1.2202733500802932E7 5808 +2.768255626449925E7 5809 +2.127349925769284E7 5810 +2.1273498604987215E7 5811 +2.768255297260165E7 5812 +5.7599627157364756E7 5813 +5.759962738028941E7 5814 +5.114352102878505E7 5815 +2.1273498915993977E7 5816 +4.5973549978197455E7 5817 +2.1273502645335834E7 5818 +3.709021245507628E7 5819 +2.7682552260045312E7 5820 +4.597354938100755E7 5821 +2.1273498923099674E7 5822 +2.7682552868863262E7 5823 +1.2202733626442617E7 5824 +9719023.105537696 5825 +9719023.290974747 5826 +5.759962758566589E7 5827 +9719023.40983485 5828 +2.1273498671108827E7 5829 +5.114352139720942E7 5830 +2.7682551535446927E7 5831 +5.1143520474473946E7 5832 +1.2202734229810294E7 5833 +4.597354899932097E7 5834 +6.103292693483384E7 5835 +2.1273501517594244E7 5836 +9719023.293050995 5837 +5.7599627150283605E7 5838 +4.5973549745425254E7 5839 +9719023.275294967 5840 +1.2202735918465022E7 5841 +5.7599627473678164E7 5842 +1.2202733888195824E7 5843 +5.7599627664195426E7 5844 +192.06911634666278 5845 +2.768255204384992E7 5846 +4.597355037337957E7 5847 +3.709021226807488E7 5848 +5.7599627390361935E7 5849 +6.103292688028636E7 5850 +8.680511316140377 5851 +2.7682552495878957E7 5852 +3.70902129700874E7 5853 +2.7682553604698293E7 5854 +5.114352052216171E7 5855 +1.2202763672260066E7 5856 +3.7090212632869735E7 5857 +121.51734874911783 5858 +5.759962843107293E7 5859 +1.2202734745815452E7 5860 +2.7682552423808143E7 5861 +2.7682551584758542E7 5862 +1.2202734074346872E7 5863 +9.703388143375342 5864 +9719029.193531806 5865 +6.103292653460724E7 5866 +4.597356322842272E7 5867 +5.114352100185375E7 5868 +6.10329284728541E7 5869 +1.2202738356153768E7 5870 +9719023.411984187 5871 +1.220273546686282E7 5872 +5.759963009245215E7 5873 +5.759962847876412E7 5874 +8.398157363903007 5875 +6.103292774796162E7 5876 +6.103292771171321E7 5877 +9719023.088967178 5878 +6.1032927107502826E7 5879 +3.3602411675112394 5880 +1.2202732981176944E7 5881 +2.1273499114357606E7 5882 +7.146289019367385 5883 +5.1143521100525714E7 5884 +9719023.383112421 5885 +4.597355023242516E7 5886 +2.127349868447614E7 5887 +2.127349947303667E7 5888 +2.768255221099926E7 5889 +4.5973549690167226E7 5890 +6.1032927116714135E7 5891 +5.759962740170242E7 5892 +9719024.299297243 5893 +2.7682552172256988E7 5894 +2.7682560306493994E7 5895 +4.597354944957485E7 5896 +62.384854585301184 5897 +1.2202733636911988E7 5898 +5.114352088557106E7 5899 +3.709021262358731E7 5900 +5.1143521000136256E7 5901 +1.2202781023998104E7 5902 +6.103292712879513E7 5903 +1.2202735018540112E7 5904 +12.05136439262016 5905 +2.127349837352736E7 5906 +1.2202734653849343E7 5907 +2.127349886068461E7 5908 +9719024.184894133 5909 +5.114352087460186E7 5910 +3.709021205933125E7 5911 +3.709021615031773E7 5912 +9719046.511164196 5913 +2.7683101086287428E7 5914 +3.709021259119297E7 5915 +9719023.010617858 5916 +2.7682551941806223E7 5917 +5.759962707384474E7 5918 +5.7599627363294534E7 5919 +2.768255208781515E7 5920 +6.103292653970096E7 5921 +4.597354923349422E7 5922 +5.114352065598732E7 5923 +2.1273499430400603E7 5924 +2.1273500889232248E7 5925 +1.2202733807898361E7 5926 +9719023.64184634 5927 +1.2202735913623638E7 5928 +2.7682552585821614E7 5929 +26.033300001072586 5930 +3.709021269580905E7 5931 +9719023.404896064 5932 +6.10329272011211E7 5933 +2.768255215083466E7 5934 +6.1032926630476646E7 5935 +5.7599629115186386E7 5936 +6.1032927851188205E7 5937 +5.114352162560341E7 5938 +3.709021381859221E7 5939 +6.103292668782202E7 5940 +1.220273388793267E7 5941 +6.1032927692005135E7 5942 +6.103292800716204E7 5943 +3.709021214816882E7 5944 +1.2202732876155714E7 5945 +2.768255216546752E7 5946 +5.114352206465624E7 5947 +4.5973551840289034E7 5948 +4.597354911473769E7 5949 +4.597354957698267E7 5950 +5.114352116094089E7 5951 +9719023.247082584 5952 +5.114352043283004E7 5953 +2.1273500725816395E7 5954 +5.7599627765589625E7 5955 +3.709021271584718E7 5956 +6.103292712626987E7 5957 +3.709021495339699E7 5958 +4.5973549173261836E7 5959 +5.114352078620982E7 5960 +115.01539782008167 5961 +2.768255234588677E7 5962 +6.103292707069649E7 5963 +45.756133962070166 5964 +2.127349962890058E7 5965 +5.114352416681389E7 5966 +9719023.11897048 5967 +5.759962994282735E7 5968 +9719023.561881803 5969 +5.114380988132263E7 5970 +2.12734984492372E7 5971 +9719025.38752567 5972 +5.759962724215966E7 5973 +4.597354887069269E7 5974 +6.771641278028966 5975 +2.7682553379220724E7 5976 +1.2202737065512504E7 5977 +2.7682551916236345E7 5978 +4.5973549929802805E7 5979 +4.597354908361966E7 5980 +1.2202733471506212E7 5981 +5.759962728521034E7 5982 +2.1273499356627434E7 5983 +4.597355036953372E7 5984 +2.7682551908035487E7 5985 +5.1143521512951575E7 5986 +16.53401698324471 5987 +9719023.192289764 5988 +8.301866290116688 5989 +5.759962737349045E7 5990 +5.759962712577176E7 5991 +4.597354903786053E7 5992 +5.11435208369332E7 5993 +6.10329726737393E7 5994 +2.1273498709014762E7 5995 +2.7682552681876015E7 5996 +2.1273499175488364E7 5997 +5.11435211297874E7 5998 +5.114352088985619E7 5999 +3.7090212698442884E7 6000 +2.768255285505371E7 6001 +5.114352147747463E7 6002 +5.1143522110295154E7 6003 +40.34266463495848 6004 +1.220273356582654E7 6005 +2.1273499135135893E7 6006 +5.114352047466604E7 6007 +6.1032927227907956E7 6008 +5.759962720437483E7 6009 +5.759962828562961E7 6010 +9719023.284322456 6011 +4.597356024926827E7 6012 +28.872258478810288 6013 +9719023.681306792 6014 +9719023.01483695 6015 +5.759962732031885E7 6016 +2.7682551967160664E7 6017 +6.1032927988090485E7 6018 +36.61974296081684 6019 +6.103292724412929E7 6020 +2.7682552909811724E7 6021 +5.759963102970588E7 6022 +2.768255213257843E7 6023 +8.888635856565607 6024 +3.70902131361316E7 6025 +18330.403008259633 6026 +3.709022051447486E7 6027 +6.103292781085037E7 6028 +2.768255474892974E7 6029 +9719024.376523897 6030 +2.127349951843555E7 6031 +2.1273498790662423E7 6032 +2.127351673381255E7 6033 +5.759962730431352E7 6034 +2.1273500352959972E7 6035 +2.7682551549407247E7 6036 +1.220273289128795E7 6037 +51.08009639612398 6038 +6.1032926555816725E7 6039 +1.2202733494163293E7 6040 +9719023.06256981 6041 +1.220273297727225E7 6042 +9719023.949179586 6043 +1086.8275838581453 6044 +1.2202736014677113E7 6045 +5.114352134992681E7 6046 +1.2202734897108503E7 6047 +1.22027336154753E7 6048 +5.114352085448553E7 6049 +1.2202733040914154E7 6050 +6.103292680967447E7 6051 +1.2202732944073148E7 6052 +2.768255338880955E7 6053 +4.597354898151442E7 6054 +1.2202735617829014E7 6055 +2.127349947227297E7 6056 +1.2202733680019533E7 6057 +9719023.450629063 6058 +1.2202734451654281E7 6059 +3.7090212740508236E7 6060 +5.7599627597078525E7 6061 +1.2202734661969526E7 6062 +6.1032927572136596E7 6063 +7.710551313249744 6064 +5.7599628249271445E7 6065 +2.1273498367315445E7 6066 +15.253497092430484 6067 +5.7599627328298986E7 6068 +5.759962702918635E7 6069 +2.1273498881704096E7 6070 +5.759962788085332E7 6071 +3.709021238911938E7 6072 +2.1273498539095532E7 6073 +2.7682552696003076E7 6074 +6.1032927221778736E7 6075 +2.1273498624640796E7 6076 +4.597354886979071E7 6077 +3.709021627813065E7 6078 +8.204632884026843 6079 +6.103292752611114E7 6080 +3.709021219351075E7 6081 +5.759962710348529E7 6082 +3.709021257953315E7 6083 +1.2202733381372266E7 6084 +2.7682552351853974E7 6085 +14.92696580599726 6086 +22.347135404612693 6087 +2.1273499100356862E7 6088 +5.990698916603779 6089 +6.490391532004272 6090 +9719024.866094792 6091 +4.597355291827111E7 6092 +9719023.129848871 6093 +9719026.083609605 6094 +1.2202733636943528E7 6095 +1.2202735158621095E7 6096 +3.709021233328403E7 6097 +5.114352080070698E7 6098 +2.768255288000025E7 6099 +1.2202746229891928E7 6100 +3.7090212171361424E7 6101 +5.759962732151631E7 6102 +1.2202734015342295E7 6103 +9719023.263857974 6104 +39.24024001405993 6105 +6.103292917194809E7 6106 +2.768255266956301E7 6107 +1.2202733551521158E7 6108 +4.597354890844674E7 6109 +5.114352113355945E7 6110 +22.644477819205466 6111 +9719023.429465681 6112 +2.7682551986729812E7 6113 +5.114352047750783E7 6114 +5.114352068829476E7 6115 +3.7090214604634136E7 6116 +5.759962708816462E7 6117 +3.709021268994905E7 6118 +41.67434432129385 6119 +3.709021400557853E7 6120 +6.103292861646205E7 6121 +5.759962722657865E7 6122 +1.2202737385817233E7 6123 +5.1143520479465865E7 6124 +3.709021325513394E7 6125 +6.525617313684707 6126 +4.597354943532599E7 6127 +5.7599627379846536E7 6128 +2.127349907056093E7 6129 +5.114352559553597E7 6130 +5.114352086030567E7 6131 +3.7090212192308456E7 6132 +2.7682552654456105E7 6133 +15.21301781208778 6134 +11.170480169710972 6135 +4.597354941789672E7 6136 +4.5973548813074015E7 6137 +4.597354963001432E7 6138 +6.103292657014467E7 6139 +14.152313778223853 6140 +5.378061248555671 6141 +3.709021240667017E7 6142 +6.1032928304989904E7 6143 +1.22027339461714E7 6144 +5.7599627483343534E7 6145 +6.103292689011148E7 6146 +2.7682552383156307E7 6147 +2.7682553104094405E7 6148 +2.7682554969012324E7 6149 +4.597354902763143E7 6150 +1.220273519689044E7 6151 +3.7090212575411566E7 6152 +2.768255272528256E7 6153 +2.1273498432996996E7 6154 +1.2202733434438845E7 6155 +3.709021238384966E7 6156 +5.759962759839779E7 6157 +6.103292690052283E7 6158 +5.11435217521259E7 6159 +6.103292677071939E7 6160 +4.597354919771783E7 6161 +6.1032927574302986E7 6162 +4.597354890802263E7 6163 +3.7090212410280876E7 6164 +1.2202733183477739E7 6165 +2.1273533127070904E7 6166 +12.267462459918049 6167 +1.2202734969538288E7 6168 +5.7599628317477725E7 6169 +4.5973549391713046E7 6170 +6.1032927278289095E7 6171 +2.1273498726685572E7 6172 +2.1273498954777293E7 6173 +6.103292765506673E7 6174 +11.430565138539245 6175 +5.114352046083978E7 6176 +5.11435208747947E7 6177 +4.59735494494275E7 6178 +5.114352070944145E7 6179 +9719023.268509157 6180 +9719023.260569813 6181 +5.11435207237377E7 6182 +5.114352113843743E7 6183 +5.7599627422348335E7 6184 +9719023.113868756 6185 +12.353599591504278 6186 +6.1032927312057495E7 6187 +6.103292654079559E7 6188 +1.2202734864465095E7 6189 +5.759962702378895E7 6190 +2.7682556278382227E7 6191 +6.10329266891935E7 6192 +231.11307994518683 6193 +2.7682552457375403E7 6194 +2.127349894010542E7 6195 +2.127349890437936E7 6196 +2.127349833782033E7 6197 +11.014852649976834 6198 +5.114352110998535E7 6199 +1.2202734133595625E7 6200 +2.768255208467956E7 6201 +6.103292670684552E7 6202 +9719023.125772716 6203 +5.114352286047639E7 6204 +9719026.61542915 6205 +2.7682553701952673E7 6206 +3.709022029567314E7 6207 +5.114352130236805E7 6208 +1.2202746344460927E7 6209 +1.2202733437535254E7 6210 +1.2202734779064778E7 6211 +5.759962747345129E7 6212 +3.709021420295669E7 6213 +4.5973550114985384E7 6214 +1.2202733058416061E7 6215 +3.7090212756471165E7 6216 +3.7090213896430895E7 6217 +9719023.301392905 6218 +5.114352086458898E7 6219 +2.7682552147173766E7 6220 +3.7090212464887485E7 6221 +2.1273498573644023E7 6222 +2.1273499102479476E7 6223 +5.1143521286974736E7 6224 +2.12734986534844E7 6225 +5.114352060629021E7 6226 +9719023.323488178 6227 +6.10329314060969E7 6228 +13.774600011657977 6229 +2.1273501211954053E7 6230 +2.7682552011848044E7 6231 +6.1032926751358025E7 6232 +3.7090212142091356E7 6233 +3.905700201097136 6234 +5.114352179459239E7 6235 +5.75996272184278E7 6236 +2.127349846617534E7 6237 +2.1273498949714776E7 6238 +2.1273500121226795E7 6239 +3.709021317545269E7 6240 +6.103292668701608E7 6241 +2.1273499293606564E7 6242 +3.709021219936726E7 6243 +5.759962718703851E7 6244 +4.5973549407301664E7 6245 +9719023.398345135 6246 +3.7090213894176796E7 6247 +5.75996272971329E7 6248 +4.320247815969981 6249 +5.114352339860194E7 6250 +6.103292747008151E7 6251 +9719023.051654713 6252 +29.397451401883433 6253 +2.768255159463913E7 6254 +2.7682552855342343E7 6255 +1.2202735087520726E7 6256 +3.709029643584047E7 6257 +6.729091159082924 6258 +1.220273301716079E7 6259 +3.8471983702277073 6260 +5.759962703308707E7 6261 +3.709021271842966E7 6262 +5.7599627130944744E7 6263 +2.76825531581875E7 6264 +5.1143520602962665E7 6265 +2.7682551951501705E7 6266 +3.709021320337079E7 6267 +2.7682551601895712E7 6268 +2.127349861693004E7 6269 +3.7626046191739904 6270 +3.597196605027058 6271 +6.103292737655339E7 6272 +5.1143520856639616E7 6273 +470.0924078824952 6274 +3.70902128305677E7 6275 +6.103292779612312E7 6276 +3.7090212062037274E7 6277 +9719023.441681886 6278 +5.759962750354887E7 6279 +1.220273565320855E7 6280 +1.2202733408509707E7 6281 +9719023.207102485 6282 +2.76825525881652E7 6283 +2.1273498979785915E7 6284 +3.709021292916876E7 6285 +2.7682552917303514E7 6286 +2.1273505807231516E7 6287 +121.63193037545206 6288 +3.709021429149188E7 6289 +1.2202734079353124E7 6290 +3.7090212404309034E7 6291 +5.1143520803260215E7 6292 +3.709021331143094E7 6293 +15.061982509865997 6294 +2.768255408829965E7 6295 +25.917763995641337 6296 +2.768255272565704E7 6297 +5.1143521923891805E7 6298 +9719023.703490868 6299 +1.2202733334951151E7 6300 +2.1273498394107588E7 6301 +1.2202734013162263E7 6302 +5.759962711082655E7 6303 +3.709021343212105E7 6304 +6.1032927373379275E7 6305 +1.2202738909482718E7 6306 +3.709021234525583E7 6307 +5.759962745870027E7 6308 +9719071.715851434 6309 +5.114352083060168E7 6310 +2.1273498456932306E7 6311 +5.759962714039794E7 6312 +9719023.333465902 6313 +2.127349835584009E7 6314 +9719023.435548903 6315 +1.2202733959022215E7 6316 +6.103292735504345E7 6317 +6.103292809776764E7 6318 +9719023.423509773 6319 +5.759962702249418E7 6320 +5.7599627649159186E7 6321 +5.1143521314435706E7 6322 +2.7682555161306817E7 6323 +1.2202733847181275E7 6324 +63.27823802855432 6325 +4.59735491800407E7 6326 +5.759962860731081E7 6327 +9719024.294767363 6328 +5.759962727490932E7 6329 +5.759962807755982E7 6330 +3.7090217236679494E7 6331 +12.306034713700974 6332 +2.1273499088072028E7 6333 +6.235000693594249 6334 +34.21868048166341 6335 +4.597354898031877E7 6336 +2.7682551785241242E7 6337 +6.1032927334275134E7 6338 +2.7682555527943246E7 6339 +5.759962704676665E7 6340 +5.114352056489361E7 6341 +2.7682552223340835E7 6342 +4.597355076945868E7 6343 +5.114352121177623E7 6344 +1.2202734588414526E7 6345 +6.103293022590913E7 6346 +1.2202734278939486E7 6347 +9719023.163632192 6348 +2.1273499024825223E7 6349 +6.103292741177925E7 6350 +5.114352108533544E7 6351 +2.768255472576165E7 6352 +4.790830810521955 6353 +12.513673280104097 6354 +3.7090216455907226E7 6355 +5.1143520596577E7 6356 +5.177054781685193 6357 +5.114354157974358E7 6358 +1.220274102062842E7 6359 +9719024.234543731 6360 +1.2202734026149077E7 6361 +5.759962731605028E7 6362 +5.114352074374277E7 6363 +3.709021271173345E7 6364 +1.2202733805283979E7 6365 +5.114352070473599E7 6366 +2.76825516377048E7 6367 +2.1273498716175433E7 6368 +2.7682553672715478E7 6369 +1.2202733442039104E7 6370 +5.1143521372853234E7 6371 +6.103292737625833E7 6372 +2.127349947476494E7 6373 +2.1273498636209473E7 6374 +5.114352112275105E7 6375 +3.709021268663676E7 6376 +5.759962722211021E7 6377 +41.55023521905434 6378 +5.759962729406389E7 6379 +2.7682554051014353E7 6380 +3.709021449278828E7 6381 +9719023.44849943 6382 +6.103292816685761E7 6383 +8.201498013043569 6384 +9719023.2808509 6385 +1.2202734498748187E7 6386 +8.516932404024555 6387 +5.759962717181029E7 6388 +5.759962703228176E7 6389 +6.10329269850912E7 6390 +6.1032927481612206E7 6391 +9719023.108664373 6392 +6.1032927228847824E7 6393 +5.114352375567792E7 6394 +47.64606402434935 6395 +5.7599628334396094E7 6396 +4.437654641088758 6397 +6.103292701106069E7 6398 +6.1032926718887635E7 6399 +9719023.610428035 6400 +2.1273501552520756E7 6401 +5.114352093615346E7 6402 +2.127349892053521E7 6403 +5.114352509537119E7 6404 +5.7599630700909436E7 6405 +5.114352456618231E7 6406 +1.2202735266940074E7 6407 +5.759962721312109E7 6408 +9719023.038993435 6409 +3.70902129335851E7 6410 +2.768255251240471E7 6411 +2.7682552785445992E7 6412 +5.759962745625594E7 6413 +3.7090212717513785E7 6414 +5.75996271719333E7 6415 +2.7682552315795742E7 6416 +9719023.277622988 6417 +5.75996270524695E7 6418 +6.103293228503624E7 6419 +4.597354968071771E7 6420 +2.127349923283268E7 6421 +6.1032938619629085E7 6422 +2.1273499349996887E7 6423 +1.2202741118819464E7 6424 +1.2202733510657977E7 6425 +78.37530462797172 6426 +5.114352133622623E7 6427 +4.5973549764962785E7 6428 +3.7090212926373295E7 6429 +5.7599650994871855E7 6430 +9719023.307187371 6431 +4.5973555018498E7 6432 +6.1032927123385504E7 6433 +2.1273498566159822E7 6434 +6.103292811791572E7 6435 +6.103292898038814E7 6436 +5.7599627144032665E7 6437 +3.70902196956651E7 6438 +5.759969209315135E7 6439 +9719023.29075558 6440 +3.709021276128807E7 6441 +11.732889871480905 6442 +2.1273500528899807E7 6443 +6.10329274863599E7 6444 +2.127349875097721E7 6445 +5.75996274933641E7 6446 +5.759962722057212E7 6447 +4.323791190461436 6448 +2.1273498644168645E7 6449 +5.114352170036807E7 6450 +2.1273498924224783E7 6451 +5.759962727909256E7 6452 +5.114352106062079E7 6453 +4.5973549781778E7 6454 +3.7090346629182905E7 6455 +4.597355028833231E7 6456 +9719023.201027183 6457 +2.7682552696631093E7 6458 +4.960425432753792 6459 +1.2202734184719E7 6460 +4.597354921982244E7 6461 +5.114352144626044E7 6462 +5.759962711662133E7 6463 +1.2202742396675434E7 6464 +4.597354925016666E7 6465 +2.7682552505430825E7 6466 +3.709021217621723E7 6467 +2.7682552590410724E7 6468 +3.709021360078359E7 6469 +6.1032926761170864E7 6470 +5.7599627283256896E7 6471 +2.1273499026110493E7 6472 +4.597354885488604E7 6473 +4.5973548820838206E7 6474 +9719023.399437573 6475 +3.7090212838501364E7 6476 +5.759962733160419E7 6477 +1.2202733649745613E7 6478 +9719023.119611412 6479 +5.2833025965332245 6480 +5.7599627090438776E7 6481 +2.7682552748857282E7 6482 +2.7682551553580504E7 6483 +1.220273312414515E7 6484 +5.759962799477089E7 6485 +5.759962730628369E7 6486 +2.127350072576892E7 6487 +5.7599627742678165E7 6488 +2.1273499339840017E7 6489 +2.1273499091121003E7 6490 +9719023.259608919 6491 +2.1273499190329485E7 6492 +2.1273498374654338E7 6493 +3.709021292604487E7 6494 +2.1273498563663073E7 6495 +9719023.360380642 6496 +33.53996031360795 6497 +5.1143525185684204E7 6498 +5.7599628328251705E7 6499 +9719023.641886365 6500 +28.83306014144198 6501 +5.114352186248559E7 6502 +3.709021270608263E7 6503 +5.1143520864232786E7 6504 +4.5973549350492194E7 6505 +2.76825518302897E7 6506 +2.1273501895246115E7 6507 +5.114352080515935E7 6508 +4.5973549665644646E7 6509 +6.1032926879366055E7 6510 +6.1032927755699106E7 6511 +2.768256416426308E7 6512 +5.759962735411838E7 6513 +5.759962742852042E7 6514 +3.7090214811552964E7 6515 +4.597354906952973E7 6516 +12.032430720767028 6517 +3.7090213524599455E7 6518 +5.7599627083724126E7 6519 +1.2202733208633645E7 6520 +3.70902122038855E7 6521 +5.7599628990395494E7 6522 +5.114352090590369E7 6523 +2.127349834897007E7 6524 +1.2202745994897502E7 6525 +9719023.266839577 6526 +4.597354887330611E7 6527 +2.12735024265634E7 6528 +4.597355005723543E7 6529 +3.709021273413438E7 6530 +5.759962733833541E7 6531 +5.114352120789719E7 6532 +3.7090212299662806E7 6533 +4.597355010518636E7 6534 +197.9130514751466 6535 +3.709021223283032E7 6536 +5.114352102878166E7 6537 +16.13557320906875 6538 +2.1273498614629395E7 6539 +3.7090212215502545E7 6540 +5.759962802301329E7 6541 +4.597355760168284E7 6542 +5.114352194735558E7 6543 +5.114352043225211E7 6544 +5.1143521178721115E7 6545 +4.5973548823330745E7 6546 +4.597354880788421E7 6547 +1.2202733663293412E7 6548 +1.220273352975959E7 6549 +2.127349853775039E7 6550 +3.709021386523934E7 6551 +5.7599627291925915E7 6552 +4.211281234526707 6553 +2.1273498396251146E7 6554 +2.1273499000552803E7 6555 +4.5973549097908445E7 6556 +5.114352416817817E7 6557 +5.759962716135181E7 6558 +2.7682553263581E7 6559 +2.768255472123897E7 6560 +1.220273360915498E7 6561 +3.709022291648902E7 6562 +5.759962765388498E7 6563 +2.7682551607774172E7 6564 +6.103292806249768E7 6565 +5.759962713451755E7 6566 +1.2202733101290032E7 6567 +6.103292730850294E7 6568 +4.59735495783286E7 6569 +33.447072879402484 6570 +4.597354937638862E7 6571 +5.114352196163653E7 6572 +4.597354910599372E7 6573 +1.2202733121571265E7 6574 +5.759962750503548E7 6575 +5.7599628441575706E7 6576 +9719032.709018884 6577 +2.7682552994174626E7 6578 +5.759962712633354E7 6579 +6.103292740491159E7 6580 +9719023.702608513 6581 +6.1032927422755755E7 6582 +5.75996298097946E7 6583 +4.5973549238892645E7 6584 +2.1273498899046537E7 6585 +2.127349890047197E7 6586 +2.768255279899199E7 6587 +5.114352107862305E7 6588 +2.1273498676138025E7 6589 +4.597355052924384E7 6590 +6.103292696806645E7 6591 +3.2791452194194624 6592 +9719023.018660085 6593 +16.051959709744494 6594 +1.220273351643602E7 6595 +1.22027331951364E7 6596 +43.39358695958582 6597 +2.1273500149287723E7 6598 +4.597354942947661E7 6599 +5.181319102245065 6600 +2.1273499022545658E7 6601 +4.642924041740595 6602 +1.2202733608384125E7 6603 +2.7682555241945736E7 6604 +5.7599627382304035E7 6605 +2.1273498458225872E7 6606 +24.33465087931271 6607 +9719027.094125113 6608 +4.597354911951391E7 6609 +2.768255184662817E7 6610 +4.036138966148909 6611 +2.1273498463121705E7 6612 +3.709021296148191E7 6613 +5.114352244890951E7 6614 +5.759962720761653E7 6615 +4.5973548834397584E7 6616 +2.1273501119641185E7 6617 +2.768255796104924E7 6618 +2.7682552378427878E7 6619 +1.2202732870406264E7 6620 +2.7682552710218955E7 6621 +2.7682554407144647E7 6622 +9719024.744455654 6623 +1.2202733568876786E7 6624 +5.114352310059388E7 6625 +2.7682558839977153E7 6626 +1.2202733225714456E7 6627 +6.1032927293684274E7 6628 +4.597354889487195E7 6629 +2.7682552489970542E7 6630 +5.7599628228662565E7 6631 +5.759962716841676E7 6632 +5.1143521410121016E7 6633 +2.7682552745089598E7 6634 +2.7682552384721823E7 6635 +2.1273498477323603E7 6636 +6.10329285115368E7 6637 +5.7599627104342245E7 6638 +9719023.267806144 6639 +2.768255215317386E7 6640 +5.759962704681948E7 6641 +4.597354938129328E7 6642 +1.2202733033066016E7 6643 +2.1273499169388153E7 6644 +5.759962741614744E7 6645 +5.114354289585305E7 6646 +2.768255183790404E7 6647 +9719023.318651417 6648 +5.114352252031906E7 6649 +3.7090212241469E7 6650 +2.7682551777903277E7 6651 +3.709021361472597E7 6652 +3.70902138289175E7 6653 +2.1273499181548975E7 6654 +32.11307887908344 6655 +6.103293285343194E7 6656 +5.1143521103761144E7 6657 +4.597354938645766E7 6658 +9719023.264986489 6659 +5.7599627907964215E7 6660 +5.114352353276335E7 6661 +5.1143521440996744E7 6662 +4.59735489887512E7 6663 +2.1273499016277228E7 6664 +4.5973549744273745E7 6665 +52.22153540459315 6666 +12.660778387639605 6667 +1.2202733892850425E7 6668 +1.2202733410989853E7 6669 +1.2202733087648192E7 6670 +2.1273498783573274E7 6671 +5.11435216189446E7 6672 +3.709021238231966E7 6673 +5.1143521585349716E7 6674 +6.10329286013257E7 6675 +9719023.077658338 6676 +3.709022831623668E7 6677 +4.597354984552645E7 6678 +6.506712494393945 6679 +3.709021299905918E7 6680 +9719023.347609095 6681 +4.597354934056066E7 6682 +2.768255467470951E7 6683 +9719023.295579303 6684 +1.2202804492668249E7 6685 +5.11435207159419E7 6686 +5.114352139851345E7 6687 +9719023.274963278 6688 +1.2202735577794127E7 6689 +3.709021619788906E7 6690 +3.709021280751684E7 6691 +4.59735493644518E7 6692 +6.103292993102511E7 6693 +2.1273500223316226E7 6694 +5.114352114836413E7 6695 +5.114352113148722E7 6696 +5.1143521658677936E7 6697 +9719023.472634818 6698 +6.1032956630394466E7 6699 +2.768255322790374E7 6700 +3.70902128815076E7 6701 +4.5973549255453E7 6702 +9719023.132227043 6703 +2.76825520811127E7 6704 +2.768255377216722E7 6705 +2.7682552979536768E7 6706 +1.2202735119281324E7 6707 +4.597354946421375E7 6708 +2.7682552346720833E7 6709 +6.103292761211309E7 6710 +1.2202733330088135E7 6711 +3.709021378022621E7 6712 +3.709021288686718E7 6713 +5.1143520580606095E7 6714 +6.103293033508727E7 6715 +2.7682552711576607E7 6716 +1.2202733795710396E7 6717 +6.664610987352219 6718 +6.1032926946815684E7 6719 +5.759962781963314E7 6720 +5.1143520830560066E7 6721 +5.7599627092302E7 6722 +4.597354891450935E7 6723 +6.103292673652669E7 6724 +5.1143520961952575E7 6725 +4.597354881593855E7 6726 +5.759962719768692E7 6727 +3.709021256064488E7 6728 +2.1273509486679234E7 6729 +1.2202733526067592E7 6730 +6.103292754564117E7 6731 +2.127349884673254E7 6732 +4.56756072160815 6733 +4.597355022002096E7 6734 +5.114352196843636E7 6735 +3.7090221729079574E7 6736 +4.597354944412467E7 6737 +3.709021252036491E7 6738 +6.103292829584128E7 6739 +5.114352305364064E7 6740 +2.1273498696883556E7 6741 +4.226367064147341 6742 +6.103292771466993E7 6743 +3.70902128193027E7 6744 +6.103292738123707E7 6745 +5.759962732111785E7 6746 +5.7696327019308145 6747 +4.597354888346253E7 6748 +9719023.215455895 6749 +2.7682604845078144E7 6750 +9719023.582749495 6751 +5.759962821562051E7 6752 +5.114352090236629E7 6753 +2.768255177109777E7 6754 +2.76825529532841E7 6755 +4.5973549406856574E7 6756 +30.56503724826748 6757 +5.114352038279301E7 6758 +1.2202733163869174E7 6759 +2.127350581033607E7 6760 +5.759962704198371E7 6761 +5.759962726322351E7 6762 +9719023.133696735 6763 +2.7682551526731223E7 6764 +9.320614237390451 6765 +4.597354946274644E7 6766 +9719023.494153902 6767 +9719023.327049557 6768 +3.709021288876352E7 6769 +5.114352064303924E7 6770 +10.236259346453638 6771 +2.7682551951690905E7 6772 +9.113124235139027 6773 +1.2202735304436736E7 6774 +1.2202736004354266E7 6775 +2.768255266987583E7 6776 +2.7682559354562875E7 6777 +1.2202733391278105E7 6778 +7.500422626982367 6779 +9719023.460261501 6780 +9719036.00823433 6781 +3.709021247702485E7 6782 +9719023.249089455 6783 +3.7090213313494995E7 6784 +2.7682551645380218E7 6785 +3.1630824085203697 6786 +4.5973554235945955E7 6787 +5.759963585324361E7 6788 +5.1143523479621194E7 6789 +2.127350023639651E7 6790 +5.11435212895185E7 6791 +5.75996282384371E7 6792 +5.11435209824981E7 6793 +5.11435205399301E7 6794 +9719023.222946279 6795 +9719023.319463793 6796 +3.709021297725989E7 6797 +3.70902132670193E7 6798 +5.114352401133399E7 6799 +5.1513435926687485 6800 +4.597354944061517E7 6801 +6.1032926872115225E7 6802 +9719023.425470576 6803 +2.1273498419407666E7 6804 +2.7682552272127867E7 6805 +2.127349953876571E7 6806 +1.2202732874250133E7 6807 +5.1143523665842585E7 6808 +2.768255201038656E7 6809 +2.768255170422179E7 6810 +6.859172294290303 6811 +2.1273499721647207E7 6812 +9719023.183719786 6813 +5.114352173783902E7 6814 +6.103292663125148E7 6815 +2.7682566019690696E7 6816 +3.709021235083953E7 6817 +9719023.390675906 6818 +2.7682552030762967E7 6819 +30.813600288048455 6820 +5.7599627703063115E7 6821 +6.1032927240016885E7 6822 +5.7599627114669986E7 6823 +2.7682552754708026E7 6824 +5.759962764636317E7 6825 +5.759962730027227E7 6826 +5.114352036542653E7 6827 +6.103292677147776E7 6828 +6.103292831815968E7 6829 +6.103292799044022E7 6830 +6.103292684041497E7 6831 +2.1273501674465243E7 6832 +2.7682553930247366E7 6833 +6.1032926666814804E7 6834 +5.759962938194819E7 6835 +2.7682552002571233E7 6836 +1.2202736757468428E7 6837 +4.597354886011228E7 6838 +2.7682553413792677E7 6839 +5.7599627312250294E7 6840 +20.618095348597713 6841 +3.70902121032375E7 6842 +5.457838767080798 6843 +2.127350255791193E7 6844 +5.759962752009009E7 6845 +5.759962790983197E7 6846 +5.1143521063459255E7 6847 +2.7682552530063264E7 6848 +2.127349884167691E7 6849 +2.7682552709802702E7 6850 +1.2202745574727366E7 6851 +9719028.887686687 6852 +5.759962741815159E7 6853 +5.759962727191841E7 6854 +4.4250494836365455 6855 +3.607255472848814 6856 +4.597355258797105E7 6857 +5.7599627213565886E7 6858 +6.103292661911609E7 6859 +3.709021332004748E7 6860 +2.1273498886375748E7 6861 +56.501267938784736 6862 +1.2202735462018223E7 6863 +7.834728214383472 6864 +3.709021259451476E7 6865 +5.759962765271471E7 6866 +1.2202733301014194E7 6867 +1.2202733825675948E7 6868 +5.759962779961159E7 6869 +9719023.038417002 6870 +2.1273498630033117E7 6871 +3.7090217767813675E7 6872 +6.1032927315223284E7 6873 +3.70902122977078E7 6874 +6.1032927442315094E7 6875 +1.2202739762593227E7 6876 +3.886313086711449 6877 +5.7599628227234356E7 6878 +5.7599627451283805E7 6879 +5.1143521742269E7 6880 +2.1273527255363334E7 6881 +2.1273498690410357E7 6882 +4.597354954668571E7 6883 +6.103292708590879E7 6884 +3.709021310361207E7 6885 +5.114352239442793E7 6886 +1.220273465984484E7 6887 +2.127352210851767E7 6888 +5.759962745361529E7 6889 +5.1143520415007845E7 6890 +1.2202736125604358E7 6891 +9719023.182295348 6892 +2.7682552903224185E7 6893 +2.768255217675241E7 6894 +2.7682553538145483E7 6895 +1.220273373511543E7 6896 +6.103292836076912E7 6897 +9719023.389611771 6898 +1.2202733747793587E7 6899 +4.5973548967123605E7 6900 +6.103292690044407E7 6901 +6.103292707654817E7 6902 +9719023.014180735 6903 +12.641257228382267 6904 +5.7599627661261395E7 6905 +2.768255546402317E7 6906 +3.7090212616063476E7 6907 +3.0539316474352063 6908 +9719024.023167802 6909 +2.1273499441215966E7 6910 +4.597355015686518E7 6911 +3.709021591535975E7 6912 +5.1143520580730595E7 6913 +9719025.84776963 6914 +1.2202735215182867E7 6915 +9719023.240370752 6916 +4.5973548992205575E7 6917 +5.114352040461601E7 6918 +9719023.314521914 6919 +5.7599627163140565E7 6920 +2.768255683928395E7 6921 +1.2202734038174342E7 6922 +5.114352075768916E7 6923 +2.7682553616270803E7 6924 +1.2202735639224814E7 6925 +9719023.207398523 6926 +3.709021301608368E7 6927 +8.71636073569868 6928 +2.7682552781609736E7 6929 +6.1032928815755E7 6930 +3.70902134576957E7 6931 +6.103292720255175E7 6932 +2.127349887972825E7 6933 +6.1032927042402625E7 6934 +4.597355155978055E7 6935 +36.30738123603306 6936 +6.103292738036919E7 6937 +6.103292698683132E7 6938 +5.114352052001536E7 6939 +5.1143520826673105E7 6940 +4.597355048245833E7 6941 +2.127349880347452E7 6942 +1.220273403990117E7 6943 +5.1143521598384455E7 6944 +6.103292680674771E7 6945 +2.1273505025826503E7 6946 +6.945235820923154 6947 +3.709021424527198E7 6948 +3.709021404386763E7 6949 +6.103292802902608E7 6950 +9719023.7018914 6951 +2.768255153620924E7 6952 +3.70902136624213E7 6953 +4.597354937159899E7 6954 +2.7682551850959312E7 6955 +2.768255455920852E7 6956 +2.768255351825695E7 6957 +5.114352082020853E7 6958 +5.759962705344418E7 6959 +5.114352070175161E7 6960 +5.759962728377663E7 6961 +3.709021307975001E7 6962 +5.1143520425347544E7 6963 +9719023.214949973 6964 +2.768255153486003E7 6965 +2.1273498462666534E7 6966 +2.768255193553111E7 6967 +2.1273498900477365E7 6968 +1.2202733308563394E7 6969 +2.1273498500874516E7 6970 +4.597354919016065E7 6971 +4.597354920501712E7 6972 +3.709021337812932E7 6973 +9719023.236847864 6974 +5.114352175535407E7 6975 +15.718818123483434 6976 +2.1273498741175782E7 6977 +5.114352151636166E7 6978 +9719027.741869645 6979 +1.2202733138464302E7 6980 +1.2202733474491384E7 6981 +4.597355160055507E7 6982 +9719023.492928054 6983 +2.1273499289609466E7 6984 +10.629038527512947 6985 +5.1143520788230464E7 6986 +4.5973548811606444E7 6987 +3.756803982950759 6988 +1.2202735333633242E7 6989 +4.597354880469221E7 6990 +6.103292752062261E7 6991 +11.683432793106338 6992 +9719023.199566577 6993 +2.127349972531302E7 6994 +6.103292715382008E7 6995 +4.597354925604623E7 6996 +9719023.03343278 6997 +9719023.127346378 6998 +5.1143522175657086E7 6999 +2.127350970152581E7 7000 +2.1273498923174623E7 7001 +5.759962759680928E7 7002 +5.114352067417588E7 7003 +3.709021456895783E7 7004 +9719023.157384096 7005 +2.1273499139010105E7 7006 +4.59735493148688E7 7007 +5.759963375823633E7 7008 +6.1032926542225614E7 7009 +6.103292772136675E7 7010 +57.18966629339576 7011 +4.597354900041665E7 7012 +9719024.535136614 7013 +1.220283543275013E7 7014 +4.5973549339298904E7 7015 +3.70902128955937E7 7016 +2.7682553414638497E7 7017 +6.103292883903654E7 7018 +9719023.324087244 7019 +2.1273499499802183E7 7020 +2.1273506017842982E7 7021 +5.759962916862414E7 7022 +6.10329275012653E7 7023 +1.2202733575915428E7 7024 +9719023.323757768 7025 +2.7682552865328334E7 7026 +20.45648079955969 7027 +6.103292655208217E7 7028 +9719023.39961035 7029 +5.759962738947109E7 7030 +1.2202736161778059E7 7031 +5.7599627224134065E7 7032 +1.22027339047726E7 7033 +2.7682553004341815E7 7034 +2.768255201010734E7 7035 +4.597355796408741E7 7036 +9719023.181726536 7037 +16.019418162600225 7038 +6.28421313461849 7039 +3.709021368983654E7 7040 +9719023.140908629 7041 +1.2202733901165659E7 7042 +4.5973549521273956E7 7043 +5.759962752693554E7 7044 +9719023.180166813 7045 +1.2202738664792249E7 7046 +3.709021402588092E7 7047 +1.2202742311815128E7 7048 +9719023.394798001 7049 +5.1143521227011956E7 7050 +1.2202762358515242E7 7051 +5.114352057407351E7 7052 +6.103292710967222E7 7053 +20.75874243473946 7054 +2.1273499086256757E7 7055 +5.759962742416068E7 7056 +9719023.203088982 7057 +2.1273503969964635E7 7058 +1.2202733137365965E7 7059 +9719023.20135981 7060 +19.784470442327383 7061 +15.48159737445238 7062 +5.114352109904487E7 7063 +4.597354930006584E7 7064 +1.2202734552284414E7 7065 +70.06609357965289 7066 +1.2202734026144126E7 7067 +51.05355857470829 7068 +5.1143535279809535E7 7069 +2.768255246618496E7 7070 +3.7090212705000095E7 7071 +2.768255250872719E7 7072 +5.759962924350119E7 7073 +4.597354899917652E7 7074 +2.1273498712899864E7 7075 +2.7682552638861097E7 7076 +5.114352962545635E7 7077 +2.1273498961663555E7 7078 +1.2202733164163506E7 7079 +3.709023049362405E7 7080 +5.114352439606268E7 7081 +6.103292692915984E7 7082 +4.59735490018372E7 7083 +5.759962789870962E7 7084 +2.127349906708266E7 7085 +4.597355055209922E7 7086 +4.597356824994548E7 7087 +1.2202733485025626E7 7088 +1.2202733653251551E7 7089 +5.759962720451646E7 7090 +6.103292665114716E7 7091 +4.597354901551293E7 7092 +5.759962720474091E7 7093 +1.2202751284993436E7 7094 +1.2202733143326743E7 7095 +3.7090214083550535E7 7096 +5.11435205314492E7 7097 +6.1032926525497176E7 7098 +5.11435214492078E7 7099 +1.2202733397282401E7 7100 +5.759962766505208E7 7101 +5.759962736631121E7 7102 +6.103292684436129E7 7103 +1.2202733657436317E7 7104 +2.7682567047369786E7 7105 +1.2202733821869606E7 7106 +9719023.24574088 7107 +12.5307265104299 7108 +6.103292748972897E7 7109 +5.75996274354385E7 7110 +3.709021648831536E7 7111 +5.759962726481704E7 7112 +2.768255931894894E7 7113 +4.597354905805931E7 7114 +4.5973548970261425E7 7115 +3.7090212289435424E7 7116 +2.7682562836608954E7 7117 +1.2202737019699998E7 7118 +3.7090212270994686E7 7119 +4.5973548941353865E7 7120 +5.759962736328386E7 7121 +7.9095727304878105 7122 +9719023.27402578 7123 +5.114352210244818E7 7124 +5.11435208028424E7 7125 +4.597355152625029E7 7126 +6.103292685727296E7 7127 +5.114355642504045E7 7128 +5.7599627178908564E7 7129 +14.640031475797983 7130 +3.709021251062467E7 7131 +2.1273498722220667E7 7132 +2.127350017361016E7 7133 +2.127349837982122E7 7134 +2.768255223130252E7 7135 +5.1143520873973206E7 7136 +2.1273498869648203E7 7137 +3.709021253337782E7 7138 +2.7682552693189155E7 7139 +1.2202733040293649E7 7140 +5.114352098996689E7 7141 +37.8580001384602 7142 +1.2202733474987062E7 7143 +1.2202733265650745E7 7144 +2.127349831728499E7 7145 +5.75996274692926E7 7146 +2.127349912032972E7 7147 +2.7682551979923874E7 7148 +3.7090216217413165E7 7149 +2.1273498301025547E7 7150 +4.5973549682173885E7 7151 +5.114352122673796E7 7152 +3.709027209170951E7 7153 +5.114352079647852E7 7154 +2.768255183348122E7 7155 +5.759962702071814E7 7156 +6.1032926773171365E7 7157 +6.103292654863387E7 7158 +4.5973549444214016E7 7159 +2.768255303606264E7 7160 +2.7682552131072894E7 7161 +1.2202732903668012E7 7162 +5.114352477668401E7 7163 +2.1273499195574455E7 7164 +2.127349902570909E7 7165 +3.7090222842324756E7 7166 +6.1032927173422284E7 7167 +3.7090212113778375E7 7168 +9719023.095735317 7169 +4.5973549993268974E7 7170 +4.5973550273181014E7 7171 +6.1032926543241546E7 7172 +5.114352096972215E7 7173 +1.2202733495674988E7 7174 +6.1034676292996936E7 7175 +2.1273500764223967E7 7176 +3.7090212653371595E7 7177 +5.114352115540111E7 7178 +1.2202733376437334E7 7179 +6.1032926531393185E7 7180 +4.597354922695912E7 7181 +3.709022587141422E7 7182 +9719023.025129309 7183 +5.7599647938303284E7 7184 +1.2202733189162435E7 7185 +9719037.596040525 7186 +3.709021386597608E7 7187 +1.2202732976661686E7 7188 +2.1273499213943165E7 7189 +9719023.111827318 7190 +4.5973552984345555E7 7191 +3.709021288345659E7 7192 +1.220273361737792E7 7193 +5.1143520964772694E7 7194 +3.709021295712578E7 7195 +5.75996281547589E7 7196 +2.1273498813894447E7 7197 +6.103292849444552E7 7198 +4.597354927777505E7 7199 +1.2202733659761813E7 7200 +1.2202733459542654E7 7201 +5.7599627376776E7 7202 +2.127350561125769E7 7203 +9719023.987106627 7204 +3.709021233937278E7 7205 +5.114352810831366E7 7206 +4.5973549096574575E7 7207 +1.2202735077058645E7 7208 +5.114352092361642E7 7209 +5.114352090197506E7 7210 +2.768255250188465E7 7211 +6.1032934024663694E7 7212 +6.1032927050139055E7 7213 +2.7682554351592895E7 7214 +1.2202733969465138E7 7215 +4.59735518705382E7 7216 +5.75996272951735E7 7217 +3.709021292910624E7 7218 +6.103292725618025E7 7219 +3.709021228267288E7 7220 +2.127349898466486E7 7221 +5.759962719533992E7 7222 +2.1273498520429373E7 7223 +4.5973548836605184E7 7224 +1.2202736925223673E7 7225 +2.1273499956817135E7 7226 +3.709021244537104E7 7227 +1.2202733549157826E7 7228 +5.114352403221675E7 7229 +3.7090212503082976E7 7230 +4.597355108194542E7 7231 +6.654579255499071 7232 +6.1032926622355424E7 7233 +9719023.160780994 7234 +2.1273499181545332E7 7235 +2.127350244831231E7 7236 +2.768255235117363E7 7237 +9.142525842255933 7238 +2.7682554190455623E7 7239 +3.709021279668129E7 7240 +6.1032926622673E7 7241 +4.597354878878169E7 7242 +1.2202733407878993E7 7243 +5.114352075597091E7 7244 +5.759962796783842E7 7245 +4.59735492007829E7 7246 +9719023.288282845 7247 +3.7090216450258955E7 7248 +2.1273498684707936E7 7249 +9719023.241926285 7250 +9719023.142632192 7251 +5.114352065440704E7 7252 +5.7599627321709625E7 7253 +9719023.0137048 7254 +3.709021247474788E7 7255 +10.235769620463849 7256 +5.114352064472219E7 7257 +4.597354912107688E7 7258 +15.010075296809962 7259 +2.7682552248597022E7 7260 +2.1273498550300334E7 7261 +3.709021625734525E7 7262 +5.7599627351985194E7 7263 +11.394894974872855 7264 +2.768255161059672E7 7265 +4.597354880327176E7 7266 +2.127349897816207E7 7267 +5.114352069051938E7 7268 +2.76825519663055E7 7269 +12.207570087866628 7270 +5.1143520736437365E7 7271 +4.597354990108864E7 7272 +6.1032926663616545E7 7273 +5.114352068075139E7 7274 +9719033.084296137 7275 +13.505888696159804 7276 +1.22027340999693E7 7277 +6.103292755063117E7 7278 +9719023.0975217 7279 +2.7682553381324742E7 7280 +6.1032927651450716E7 7281 +5.11435217341374E7 7282 +3.709021461578268E7 7283 +1.2202733834930925E7 7284 +4.597355001854308E7 7285 +4.5973548937854744E7 7286 +1.220273489764569E7 7287 +5.7599627235690266E7 7288 +1.2202735474500395E7 7289 +2.1273503319992844E7 7290 +2.7682552745328218E7 7291 +5.114352043077865E7 7292 +2.1273500372222908E7 7293 +5.114352117803775E7 7294 +1.2202733494893802E7 7295 +27.19636512548357 7296 +2.7682552558742646E7 7297 +5.114352820166648E7 7298 +13.47151190235903 7299 +1.2202733153474413E7 7300 +9719023.055884663 7301 +6.103292693066904E7 7302 +9719023.018968632 7303 +2.1273498512327634E7 7304 +4.5973549378739074E7 7305 +2.768255389920445E7 7306 +1.2202733343536954E7 7307 +5.759963138002357E7 7308 +2.7682552897994235E7 7309 +134.45879965533192 7310 +9.27220372989811 7311 +5.114352117880948E7 7312 +3.709021424818586E7 7313 +4.597354933031544E7 7314 +12.322874016733886 7315 +9719023.14733075 7316 +4.597355057292888E7 7317 +4.5973548921685636E7 7318 +2.1273499590955086E7 7319 +3.709021288104415E7 7320 +4.597354938420505E7 7321 +5.759962725696599E7 7322 +3.709021290523162E7 7323 +3.709021279590302E7 7324 +2.7682552477159012E7 7325 +5.114352087106338E7 7326 +6.10329273573749E7 7327 +3.709028764356674E7 7328 +3.709021241416595E7 7329 +2.127349836501084E7 7330 +2.768255161365421E7 7331 +1.2202733380150858E7 7332 +9719023.220660457 7333 +2.127349844216927E7 7334 +5.1143520975938424E7 7335 +10.08341413682721 7336 +1.2202734150044063E7 7337 +3.7090212822107896E7 7338 +2.12734984253174E7 7339 +4.597354882617411E7 7340 +3.709021247392996E7 7341 +5.1143520616966315E7 7342 +4.423270895609488 7343 +5.759962738186825E7 7344 +6.103293099033779E7 7345 +5.11435209032753E7 7346 +9719023.565032482 7347 +1.2202735492822599E7 7348 +9719023.623957206 7349 +4.251828101971492 7350 +6.1032927749423824E7 7351 +5.114352056051709E7 7352 +9719023.30230641 7353 +8.975120195303733 7354 +6.103292778698979E7 7355 +3.7090213675435804E7 7356 +2.768255248282436E7 7357 +4.5973551219229475E7 7358 +4.597355049445805E7 7359 +9719023.20439884 7360 +2.127349862976381E7 7361 +1.2202733215080885E7 7362 +1.2202756372293744E7 7363 +6.1032927241132446E7 7364 +2.7682552801542804E7 7365 +6.103292697772416E7 7366 +9719026.487377698 7367 +5.759962729811834E7 7368 +4.597354951483889E7 7369 +9.5959010911738 7370 +25.18298382456366 7371 +2.127350043437721E7 7372 +6.103292713946486E7 7373 +3.709021204730732E7 7374 +2.7682551735453416E7 7375 +12.09439873582102 7376 +5.114352067980561E7 7377 +292.17953442251655 7378 +4.5973549530412935E7 7379 +3.709021359693993E7 7380 +21.318395527863238 7381 +5.114352175090731E7 7382 +2.1273499589435134E7 7383 +1.2202733374667188E7 7384 +6.103292661561372E7 7385 +9719023.15030138 7386 +3.709021284582776E7 7387 +2.7682552221201316E7 7388 +2.127350200893962E7 7389 +5.114352168962421E7 7390 +2.1273500582117517E7 7391 +2.127400008198971E7 7392 +6.103293527811901E7 7393 +6.103292658184058E7 7394 +4.491768466267032 7395 +3.709022093259913E7 7396 +2.1273498983400006E7 7397 +6.103292706355299E7 7398 +2.7682554827450138E7 7399 +6.103292723968785E7 7400 +5.759962735961989E7 7401 +5.1143521488181315E7 7402 +6.103292811800418E7 7403 +4.597354899429723E7 7404 +3.709021481074748E7 7405 +5.114352115843419E7 7406 +6.103293146172151E7 7407 +6.1032926564129695E7 7408 +2.1273507800450835E7 7409 +5.759962717900572E7 7410 +5.114352077631558E7 7411 +9719023.542969445 7412 +10.615512054911648 7413 +1.2202737181017071E7 7414 +4.597354902432356E7 7415 +2.127349933862815E7 7416 +6.1032929179696545E7 7417 +21.5731663869438 7418 +2.7682566291255232E7 7419 +6.1032926735914014E7 7420 +9719024.019470261 7421 +9719023.3161965 7422 +3.709021205631161E7 7423 +9719023.777436255 7424 +5.114352073378622E7 7425 +5.7599627162455395E7 7426 +2.1273503178812068E7 7427 +2.768255542638317E7 7428 +2.7682551995221972E7 7429 +5.759963055433079E7 7430 +2.7682551522374347E7 7431 +6.592919938903557 7432 +4.597354950174652E7 7433 +6.103292672836446E7 7434 +6.10329277429169E7 7435 +58.905106164808004 7436 +3.709021224087209E7 7437 +12.899383792741753 7438 +9719023.316232884 7439 +9719023.101691825 7440 +6.10329314578689E7 7441 +5.759962793129447E7 7442 +2.127349909157427E7 7443 +9719025.936249102 7444 +9719023.588380963 7445 +5.759963346470574E7 7446 +5.7599630442380324E7 7447 +2.1273498391146634E7 7448 +4.597355095549863E7 7449 +9719023.218613194 7450 +2.768256200299248E7 7451 +3.6527852856229863 7452 +4.597354922135136E7 7453 +3.709021450893224E7 7454 +2.1273498532233603E7 7455 +5.7599627322369635E7 7456 +5.759962807708708E7 7457 +4.597354880453167E7 7458 +1.2202733650944274E7 7459 +5.1143522066245586E7 7460 +1.2202734853340168E7 7461 +1.2202736091328114E7 7462 +3.709021204131224E7 7463 +3.709021249817695E7 7464 +4.5973549101757914E7 7465 +4.597354879926348E7 7466 +3.709021283107413E7 7467 +9719023.085184114 7468 +4.5973549381495304E7 7469 +5.20534233699182 7470 +16.276340980617924 7471 +1.2202732947879156E7 7472 +9719026.116707811 7473 +2.1273498531784493E7 7474 +2.7682554171923332E7 7475 +8.564912409086313 7476 +3.709021248181077E7 7477 +11.487873901612042 7478 +5.114352180953775E7 7479 +5.7599629179358646E7 7480 +9719023.299640564 7481 +5.114352501024271E7 7482 +2.768255173733568E7 7483 +2.7682564739213355E7 7484 +9719023.056464968 7485 +6.103292789605382E7 7486 +9719023.171581998 7487 +1.220273416660029E7 7488 +2.1273498483464666E7 7489 +2.7682551922845196E7 7490 +4.597354888793488E7 7491 +59.30009783904011 7492 +9719023.196793811 7493 +3.709021213433627E7 7494 +9719025.402513018 7495 +9719026.231957786 7496 +9719023.088407457 7497 +4.52759666633402 7498 +6.103292717226095E7 7499 +3.7090212966261424E7 7500 +4.597355033034652E7 7501 +16.64250353459568 7502 +5.7599627631311685E7 7503 +9719023.640011385 7504 +3.709022417084715E7 7505 +4.597354967524437E7 7506 +5.7599627133348E7 7507 +1.2202733220728537E7 7508 +2.768255155779958E7 7509 +4.565791505330766 7510 +9719023.04957064 7511 +4.597354898027753E7 7512 +4.5973549076713294E7 7513 +9719023.071405297 7514 +2.7682553200735535E7 7515 +9719024.455700643 7516 +2.1273500778766878E7 7517 +9719023.033908214 7518 +1.220273297443244E7 7519 +9719023.49288153 7520 +6.10329270888702E7 7521 +2.1273499778826956E7 7522 +3.709021386376856E7 7523 +5.759962717389748E7 7524 +5.7599627077031866E7 7525 +4.5973549262092255E7 7526 +2.1273501351692457E7 7527 +3.709021245332576E7 7528 +6.1032928384569064E7 7529 +3.7090212597982235E7 7530 +9719023.141862538 7531 +3.7090212532227896E7 7532 +5.7599627235665604E7 7533 +4.597354950328791E7 7534 +9719023.821716378 7535 +9719023.305572638 7536 +5.1143524010902666E7 7537 +2.1273499477522627E7 7538 +5.1143520561337665E7 7539 +1.2202736091779264E7 7540 +6.208404358717921 7541 +1.220273346629963E7 7542 +2.1273498654153585E7 7543 +9719023.234549858 7544 +2.768255223583111E7 7545 +6.103292665694703E7 7546 +4.597354933019991E7 7547 +5.1143520593805015E7 7548 +2.1273498859563638E7 7549 +1.220273370322374E7 7550 +4.59735490872571E7 7551 +3.63477649240824 7552 +9.504700964270762 7553 +9719023.164299464 7554 +5.1143521191383414E7 7555 +5.759962836406552E7 7556 +9719023.288924947 7557 +1.2202733827097E7 7558 +4.59735521110439E7 7559 +5.759962732972583E7 7560 +4.597355291784085E7 7561 +9719023.120650657 7562 +4.5973549313285045E7 7563 +1.220273387095041E7 7564 +6.1032929102818914E7 7565 +6.103292827958505E7 7566 +6.103292667306567E7 7567 +1.220273317880509E7 7568 +3.709021223027147E7 7569 +6.103292651686614E7 7570 +6.103292705973732E7 7571 +5.1143520596992485E7 7572 +1.2202735000126548E7 7573 +1.2202735140907135E7 7574 +5.75996277380218E7 7575 +4.597354946140329E7 7576 +5.2947578286698 7577 +6.10329277732665E7 7578 +2.7682552434844784E7 7579 +6.1032928756505005E7 7580 +2.1273500522412244E7 7581 +5.1143520534314975E7 7582 +9719023.189066108 7583 +3.709021273592674E7 7584 +1.2202734404418973E7 7585 +2.1273499612387717E7 7586 +5.1143520617441446E7 7587 +4.597355059997786E7 7588 +3.709021276076387E7 7589 +9719023.403366933 7590 +4.597354897031144E7 7591 +6.103293057811932E7 7592 +2.1273500248060063E7 7593 +1.2202733523609348E7 7594 +2.7682552386004455E7 7595 +1.2202733487808894E7 7596 +5.759962729921788E7 7597 +2.7682551786435083E7 7598 +3.709188492705659E7 7599 +5.1143520801826544E7 7600 +2.7682552223900598E7 7601 +5.7599627319717444E7 7602 +5.7599627362507425E7 7603 +9719023.061910337 7604 +2.1273499797300693E7 7605 +9719023.318663536 7606 +5.114352271438994E7 7607 +4.597354919849898E7 7608 +9719023.054909335 7609 +5.11435227708761E7 7610 +1.2202737780755062E7 7611 +2.1273498606926836E7 7612 +5.1143520746536456E7 7613 +4.597354897593195E7 7614 +3.7090212524342895E7 7615 +2.76825530224581E7 7616 +2.127350160320546E7 7617 +5.114352076687637E7 7618 +6.103292652905013E7 7619 +3.709021343176029E7 7620 +9719023.08850776 7621 +1.2202735372245753E7 7622 +1.2202734285853688E7 7623 +2.1273500699006733E7 7624 +4.597354889149305E7 7625 +1.220273356461808E7 7626 +1.2202733132961068E7 7627 +5.114352042924501E7 7628 +5.759962738047579E7 7629 +9719023.188488163 7630 +1.2202747618322503E7 7631 +5.136632873975109 7632 +1.220273338594796E7 7633 +4.373587052949434 7634 +2.1273504101704337E7 7635 +4.590742270148319 7636 +3.709021257944689E7 7637 +5.114352167434546E7 7638 +5.1143520866221584E7 7639 +9719023.42349804 7640 +5.114352207081079E7 7641 +2.127349860530767E7 7642 +9719023.328321883 7643 +5.759962708207343E7 7644 +5.114352071001233E7 7645 +9719023.446992513 7646 +4.597355096473595E7 7647 +9719023.166384047 7648 +2.1273503989451244E7 7649 +2.7682551921422616E7 7650 +3.70902120860442E7 7651 +4.5973552277975984E7 7652 +9719023.024497129 7653 +3.5133347696601835 7654 +5.759962726847614E7 7655 +5.1143522965376355E7 7656 +5.114352135342359E7 7657 +1.2202732973516615E7 7658 +9719023.285562057 7659 +2.768255379760634E7 7660 +1.220273287902953E7 7661 +7.01983607221159 7662 +4.5973549290494375E7 7663 +13.29276235839481 7664 +2.127349899061816E7 7665 +2.127349888188666E7 7666 +5.759962713582396E7 7667 +9719023.17785675 7668 +5.114352282904816E7 7669 +2.7682553088252403E7 7670 +6.103292681416925E7 7671 +2.1273498517730985E7 7672 +2.1273498462329965E7 7673 +2.1273498951873813E7 7674 +2.127350095107556E7 7675 +5.759962743137379E7 7676 +2.768255258596563E7 7677 +6.103292659934698E7 7678 +2.7682552146253355E7 7679 +2.7682552169450145E7 7680 +3.709021299513246E7 7681 +2.7682552562836293E7 7682 +2.768255267687022E7 7683 +5.759962711500848E7 7684 +5.7599627147159226E7 7685 +6.1032935254547074E7 7686 +4.597354987302285E7 7687 +6.103292741237679E7 7688 +9.775151753834294 7689 +1.2202733045411898E7 7690 +5.759962708579223E7 7691 +6.103292654707928E7 7692 +3.70902140441196E7 7693 +2.7682551842185516E7 7694 +9719023.08149234 7695 +1.2202734616193827E7 7696 +5.759962763320501E7 7697 +1.2202733203912279E7 7698 +8.53549234486394 7699 +66.54775556326251 7700 +2.7682551815385632E7 7701 +6.103292754537783E7 7702 +5.1143527497838736E7 7703 +2.1273499662185088E7 7704 +14.195183617862748 7705 +1.2202733926446302E7 7706 +4.597354941026412E7 7707 +3.709021303406505E7 7708 +4.59735505897625E7 7709 +1.2202735615045642E7 7710 +1.2202733652765427E7 7711 +4.597354906385131E7 7712 +9719023.493244197 7713 +36.19303891000567 7714 +2.1273500147552982E7 7715 +2.127349831901176E7 7716 +6.715686653668543 7717 +6.103292701818942E7 7718 +5.759962741450794E7 7719 +2.127349895173014E7 7720 +2.1273498793498818E7 7721 +5.7599627153357446E7 7722 +58.77700115205715 7723 +4.597354906600451E7 7724 +75.26796102102583 7725 +4.597354937391639E7 7726 +2.7682551688189078E7 7727 +22.672741572869935 7728 +3.709021451643263E7 7729 +23.44883431025514 7730 +1.2202733093907427E7 7731 +4.597358152469835E7 7732 +4.597354939059145E7 7733 +2.768255437871488E7 7734 +32.990545343823406 7735 +8.989268278290906 7736 +5.759963229347579E7 7737 +3.709021203816397E7 7738 +2.7682552763116315E7 7739 +5.114352079546552E7 7740 +5.1143524741564944E7 7741 +4.597355067177892E7 7742 +5.75996272220237E7 7743 +14.458805042638774 7744 +2.127349833432464E7 7745 +1.2202737684843315E7 7746 +6.103292710839583E7 7747 +1.2202734496247848E7 7748 +3.709021240874425E7 7749 +1.2202733092372024E7 7750 +33.718039621804365 7751 +5.114352091855819E7 7752 +1.2202733931465171E7 7753 +1.2202733263687292E7 7754 +4.597354933755997E7 7755 +2.1273498893238507E7 7756 +1.2202743490674E7 7757 +6.103292770889176E7 7758 +104.26895656620623 7759 +1.2202733608774079E7 7760 +7.689621861839474 7761 +2.7682557846829973E7 7762 +3.709021287353586E7 7763 +4.597354887514106E7 7764 +4.597354923299004E7 7765 +10.63790197506686 7766 +5.759962754937498E7 7767 +9719023.350075502 7768 +2.1273500425542146E7 7769 +5.114352093950557E7 7770 +9719023.037511634 7771 +6.1032928031703316E7 7772 +1.2202733301213669E7 7773 +7.335682917175298 7774 +5.759962742451547E7 7775 +3.709021390839428E7 7776 +6.103292665288924E7 7777 +9719023.212134626 7778 +2.1273499060199548E7 7779 +5.759962718140447E7 7780 +4.597354926128702E7 7781 +9719023.02313948 7782 +3.709021244035516E7 7783 +9719023.740022408 7784 +6.1032952546552956E7 7785 +3.709021222867447E7 7786 +9719023.223592615 7787 +2.7682551912417296E7 7788 +6.1032926699205436E7 7789 +72.05545261158981 7790 +3.709021251763604E7 7791 +2.127349909826319E7 7792 +4.5973551706276365E7 7793 +1.2202746400003396E7 7794 +5.759962716866941E7 7795 +6.103293095110009E7 7796 +9719023.709618967 7797 +9719023.680108963 7798 +1.2202732938512815E7 7799 +9719023.42723762 7800 +2.7682551576952375E7 7801 +5.114352122715451E7 7802 +2.127349931883057E7 7803 +5.759963014227285E7 7804 +6.103292657480729E7 7805 +1.2202733345483467E7 7806 +4.597354903232942E7 7807 +118.15635549311207 7808 +1.2202733507896105E7 7809 +9719023.063178213 7810 +1.2202734815027615E7 7811 +22.944481989413845 7812 +9719023.288851319 7813 +20.908795510577526 7814 +5.759962774410589E7 7815 +4.597354928663138E7 7816 +3.709021287184976E7 7817 +9719023.044564648 7818 +6.695886226144426 7819 +6.103292766198461E7 7820 +5.759962741246361E7 7821 +9719025.270379473 7822 +6.10329265487059E7 7823 +3.0107117776496475 7824 +5.759962721164762E7 7825 +5.114352119957122E7 7826 +3.709021369032641E7 7827 +3.709021355205775E7 7828 +2.1273498745662645E7 7829 +2.768255215160589E7 7830 +2.7682552452034112E7 7831 +5.114352463755861E7 7832 +2.127349870020023E7 7833 +1.2202733203331152E7 7834 +3.7090212445619635E7 7835 +6.1032927525259E7 7836 +6.103292717392376E7 7837 +5.1143524408212624E7 7838 +4.59735490649497E7 7839 +2.768255272725986E7 7840 +2.1273500009880275E7 7841 +9719023.112689953 7842 +5.759963633463085E7 7843 +9719034.292632014 7844 +2.1273498572628185E7 7845 +1.2202739655887952E7 7846 +4.5973548842129394E7 7847 +2.1273499239534322E7 7848 +10.425847693791718 7849 +6.340550608034335 7850 +3.7090213358764164E7 7851 +5.712182049456852 7852 +6.103292742802961E7 7853 +2.7682551634125195E7 7854 +5.759962711024202E7 7855 +5.114352186133469E7 7856 +6.901360758334123 7857 +2.768255221542167E7 7858 +5.759962884160755E7 7859 +1.2202735333849298E7 7860 +5.114352056989198E7 7861 +3.7090212073697045E7 7862 +6.103292946171065E7 7863 +9719023.187617557 7864 +50.82557859230942 7865 +3.7090212427210085E7 7866 +4.597355276774022E7 7867 +2.1273498352136627E7 7868 +4.5973549430084035E7 7869 +6.103293469180103E7 7870 +2.768255283904662E7 7871 +3.709021308470703E7 7872 +1.2202733706184983E7 7873 +9719023.019995866 7874 +2.1273498935276452E7 7875 +5.759962862377878E7 7876 +3.709021214843465E7 7877 +4.215985923624463 7878 +5.1143523741651244E7 7879 +9.222704875345364 7880 +2.7682552157958005E7 7881 +9719023.114490094 7882 +5.759962744353735E7 7883 +1.2202733128220432E7 7884 +2.127349891760518E7 7885 +9719023.745169535 7886 +3.709021392133453E7 7887 +4.597355244029833E7 7888 +6.103292651154041E7 7889 +5.114352525451816E7 7890 +2.768255182062764E7 7891 +2.1273499424768213E7 7892 +2.127350017459419E7 7893 +3.709021328062782E7 7894 +2.7682552641671907E7 7895 +2.1273500606693175E7 7896 +5.759962779056275E7 7897 +3.709021210124395E7 7898 +2.127349879586363E7 7899 +2.1273498401969243E7 7900 +1.2202733732278598E7 7901 +3.709021378227977E7 7902 +4.5973548811450295E7 7903 +1.2202733717941266E7 7904 +3.7090215256177954E7 7905 +1.220273383375963E7 7906 +2.7682552734125316E7 7907 +6.47535319230743 7908 +79.40948989184402 7909 +1.2202740154050427E7 7910 +3.709021475691598E7 7911 +85.4400473379722 7912 +3.709021533423856E7 7913 +1.2202733359725969E7 7914 +5.114352071709145E7 7915 +3.7090212943642974E7 7916 +1.2202733562899927E7 7917 +4.597354943811779E7 7918 +4.5973548955914706E7 7919 +5.114372761077568E7 7920 +1.2202733934029475E7 7921 +4.597355072259054E7 7922 +9719023.236105649 7923 +2.1273498855320793E7 7924 +6.1032926971002445E7 7925 +5.7599627162742406E7 7926 +9719023.121000508 7927 +5.759962716007353E7 7928 +2.7682552113597173E7 7929 +3.709021204519628E7 7930 +4.5973554302908264E7 7931 +5.759962720191756E7 7932 +4.597355001170916E7 7933 +6.103292738968497E7 7934 +1.2202733674220627E7 7935 +6.0426662351641385 7936 +5.114352086793492E7 7937 +4.597354929394434E7 7938 +5.759962738424061E7 7939 +1.220273451174161E7 7940 +2.1273498917089827E7 7941 +4.527139900395058 7942 +4.5973550188964166E7 7943 +6.103292696676825E7 7944 +4.3876609696494 7945 +7.082483273825145 7946 +5.759962701881877E7 7947 +5.75996274921369E7 7948 +6.1032927011883266E7 7949 +2.1273498851845004E7 7950 +2.1273498841184326E7 7951 +2.768255175973698E7 7952 +1.2202732877784776E7 7953 +2.7682553955979414E7 7954 +9719023.093447292 7955 +6.1032927088610634E7 7956 +5.75996272124091E7 7957 +6.103292681211736E7 7958 +5.114352083219275E7 7959 +3.709021332126567E7 7960 +6.103292692176969E7 7961 +6.10329272324019E7 7962 +5.114352066237774E7 7963 +3.709021259101767E7 7964 +4.5973548976483956E7 7965 +17.08671279141565 7966 +4.597354909595616E7 7967 +2.127350327167718E7 7968 +4.5973553406512894E7 7969 +9719023.074313061 7970 +6.103292934932844E7 7971 +5.759963122143843E7 7972 +9719023.029728424 7973 +6.103292658358939E7 7974 +4.5973549275422245E7 7975 +5.759962711633037E7 7976 +3.7090212478067726E7 7977 +2.7682551872107547E7 7978 +3.709021272512329E7 7979 +1.2202733871353483E7 7980 +5.1143520755619034E7 7981 +5.759962782331778E7 7982 +2.7682552052821588E7 7983 +6.899286476204144 7984 +2.1273498583020527E7 7985 +4.597355328441288E7 7986 +3.709021307249489E7 7987 +6.1032927386627354E7 7988 +2.768255185641708E7 7989 +6.103292921909935E7 7990 +2.127349834077911E7 7991 +4.597354945769073E7 7992 +6.103292700185778E7 7993 +3.709021241332604E7 7994 +4.5973549137733914E7 7995 +3.709021426727205E7 7996 +1.2202733243460484E7 7997 +3.709021641131642E7 7998 +2.7682552049848005E7 7999 +3.709021217348956E7 8000 +4.597354931576428E7 8001 +2.127350273703512E7 8002 +4.597354953658592E7 8003 +1.2202733266264124E7 8004 +4.597354881804063E7 8005 +3.709021301885787E7 8006 +2.7682553455882005E7 8007 +9719023.10549443 8008 +8.590871339216967 8009 +9719023.255347757 8010 +25.500853038552613 8011 +3.709021380841018E7 8012 +4.5973549025092766E7 8013 +2.7682551866992135E7 8014 +1.2202733410140356E7 8015 +5.114352099844787E7 8016 +2.127349890634069E7 8017 +6.103292761909456E7 8018 +9719023.371044595 8019 +4.597354951353893E7 8020 +9719023.20686253 8021 +5.114352051445202E7 8022 +5.759963138492155E7 8023 +9.041966677453509 8024 +5.1143523726364106E7 8025 +4.597354973450884E7 8026 +5.7599627480884E7 8027 +4.597355325986514E7 8028 +29.957422196255767 8029 +2.1273499852691997E7 8030 +7.007063677259557 8031 +5.1143520835044794E7 8032 +5.114352105855752E7 8033 +3.709021233417545E7 8034 +5.114352119320139E7 8035 +5.75996272614177E7 8036 +1.2202733655681565E7 8037 +2.127349895137946E7 8038 +4.597354935645282E7 8039 +5.114352164012197E7 8040 +3.709021299095748E7 8041 +4.5973549105651565E7 8042 +2.1273500208449572E7 8043 +142.77308823973357 8044 +14.115793470205482 8045 +5.114352042624762E7 8046 +5.759962734783295E7 8047 +6.1032928695932604E7 8048 +3.709021241023382E7 8049 +3.7090216054226704E7 8050 +6.103293064529332E7 8051 +6.103293535148531E7 8052 +5.759962742857928E7 8053 +2.1273498813363515E7 8054 +4.597355127992157E7 8055 +5.1143520377368465E7 8056 +6.1032928668567896E7 8057 +5.75996272147129E7 8058 +9.544130783320586 8059 +9719024.021975772 8060 +5.114352123301635E7 8061 +1.2202733292079985E7 8062 +12.2475134620601 8063 +6.103292661725911E7 8064 +5.759962853411636E7 8065 +4.597354907354484E7 8066 +9719023.285528066 8067 +1.2202734352728246E7 8068 +4.597354928288755E7 8069 +12.688265094916298 8070 +9719047.149901174 8071 +2.7682553059918795E7 8072 +1.220482600902664E7 8073 +5.759962703334967E7 8074 +9719023.173475044 8075 +1.22027352063111E7 8076 +1.2202733329274446E7 8077 +2.1273501269213658E7 8078 +4.887746318514367 8079 +17.1729324314433 8080 +6.1032927606432185E7 8081 +5.7599627907175705E7 8082 +6.103292709412402E7 8083 +2.127349865950639E7 8084 +5.114352090233822E7 8085 +1.220273303588197E7 8086 +3.709021222645471E7 8087 +6.103292946502951E7 8088 +5.7599661970567726E7 8089 +4.5973552242425315E7 8090 +2.1273499699043546E7 8091 +4.597354925182151E7 8092 +5.114352120608292E7 8093 +2.7682558051453337E7 8094 +2.7682551896925278E7 8095 +2.7682551856290035E7 8096 +2.127349870773077E7 8097 +5.759962786108358E7 8098 +4.5973548870951116E7 8099 +4.597355159393676E7 8100 +5.114378276637704E7 8101 +4.597354905303655E7 8102 +1.2202733208838448E7 8103 +9719023.537416162 8104 +5.11435423244175E7 8105 +2.1273498581201915E7 8106 +1.2202741245714445E7 8107 +4.597354905512851E7 8108 +3.709021295640101E7 8109 +5.7599627403257854E7 8110 +9.83233241300855 8111 +3.709021360146205E7 8112 +9719023.174576094 8113 +5.114352071067319E7 8114 +6.103292712041219E7 8115 +1.2202733732368546E7 8116 +2.127349836171042E7 8117 +5.114352144437999E7 8118 +3.709021296742545E7 8119 +23.245055245237744 8120 +2.7682553520262998E7 8121 +2.7682554001294576E7 8122 +6.103292880308306E7 8123 +3.709021326179956E7 8124 +3.709021260526695E7 8125 +3.7090212408233464E7 8126 +9719023.22786366 8127 +2.7682553700519606E7 8128 +5.114352118297699E7 8129 +5.759962732487753E7 8130 +1.2202734805217596E7 8131 +3.70902144800307E7 8132 +4.597355113123397E7 8133 +5.759962741588815E7 8134 +9719024.044472504 8135 +4.597354880688239E7 8136 +4.59735498360187E7 8137 +5.114352077383129E7 8138 +2.768255364529136E7 8139 +9.897441736146488 8140 +9719023.702988448 8141 +3.709021234645611E7 8142 +4.597354883900685E7 8143 +2.7682552969118282E7 8144 +3.709021330619973E7 8145 +2.768255261088454E7 8146 +1.2202733844211632E7 8147 +3.7090219181072034E7 8148 +4.5973549111581095E7 8149 +5.759962968282264E7 8150 +9719024.144139046 8151 +5.759962711633353E7 8152 +9719023.259142121 8153 +20.229053773986532 8154 +2.7682552210911077E7 8155 +26.620972079950743 8156 +2.1273499875802793E7 8157 +1.2202733755616074E7 8158 +5.759962724165213E7 8159 +1.2202732975472316E7 8160 +5.1143530563089035E7 8161 +2.768257154515543E7 8162 +6.103292708673145E7 8163 +5.1143520802698754E7 8164 +18.852686464690002 8165 +2.1273498454977784E7 8166 +6.137464630618542 8167 +4.59735489779324E7 8168 +5.759962796829689E7 8169 +2.76825524322032E7 8170 +3.7785209028731304 8171 +5.1143520576845914E7 8172 +2.127349874407811E7 8173 +2.7682551885807052E7 8174 +6.103292728689475E7 8175 +2.76825549088025E7 8176 +6.103292955230266E7 8177 +2.1273498821559824E7 8178 +8.370051437437185 8179 +6.103292965105749E7 8180 +2.127349833601739E7 8181 +2.1273498302974794E7 8182 +3.709021603220044E7 8183 +2.1273515934104484E7 8184 +1.2202733763905901E7 8185 +5.7599627165001296E7 8186 +9719023.577112623 8187 +3.709021478111987E7 8188 +5.7599633130620256E7 8189 +9719023.391143313 8190 +2.127349882099752E7 8191 +2.127349902588062E7 8192 +9719023.238019086 8193 +2.768255223168719E7 8194 +5.1143520611316234E7 8195 +3.709021212851163E7 8196 +2.7682552566945534E7 8197 +2.7682566654478166E7 8198 +3.709021328942213E7 8199 +5.759962747477754E7 8200 +4.5973548941050306E7 8201 +3.709021238758021E7 8202 +3.9421605126157924 8203 +3.709021386414725E7 8204 +5.759962848180449E7 8205 +2.7682555184621617E7 8206 +6.1032928328772806E7 8207 +2.7682553221436776E7 8208 +2.1273498612316348E7 8209 +9719023.169826468 8210 +5.1143520990386106E7 8211 +5.1143522651395164E7 8212 +5.759962727682497E7 8213 +2.1273635201114178E7 8214 +6.103292762002075E7 8215 +1.2202732983360143E7 8216 +9719023.261298593 8217 +6.103292678134366E7 8218 +1.2202736178904558E7 8219 +5.759962906211444E7 8220 +4.597354934029054E7 8221 +2.1273498802241936E7 8222 +4.579574955006109 8223 +2.1273498983769037E7 8224 +1.2202735186859928E7 8225 +5.114352103957365E7 8226 +6.1032926941419326E7 8227 +5.114352547900333E7 8228 +1.2202733790301949E7 8229 +5.759962743509507E7 8230 +6.103298073892466E7 8231 +6.103292785728861E7 8232 +4.597354884719211E7 8233 +5.759962711798109E7 8234 +6.1032926670532174E7 8235 +5.7599627660832874E7 8236 +5.1143520718219206E7 8237 +2.1273500022967573E7 8238 +2.7682563225467525E7 8239 +2.7682552020579148E7 8240 +3.709021284110512E7 8241 +2.1273498633417606E7 8242 +4.5973549695176855E7 8243 +6.1032927541186415E7 8244 +32.03165214270566 8245 +2.768255154421682E7 8246 +5.114352120693988E7 8247 +2.127350040021277E7 8248 +9719023.39922484 8249 +2.768255276728736E7 8250 +6.1032927234245375E7 8251 +5.1143521205650754E7 8252 +2.127349845496054E7 8253 +1.2202733062479528E7 8254 +1.2202733044160362E7 8255 +2.7682584909388535E7 8256 +9719023.112702968 8257 +5.114352174124235E7 8258 +5.759962716160125E7 8259 +1.2202741577581583E7 8260 +2.7682552607534163E7 8261 +2.7682551668723263E7 8262 +5.114352127605418E7 8263 +2.127349953870504E7 8264 +5.759962784280626E7 8265 +5.114520603247162E7 8266 +1.2202734538694091E7 8267 +2.7682551768662836E7 8268 +3.7090213031535186E7 8269 +5.900880066287478 8270 +6.103292779350831E7 8271 +10.914800055469838 8272 +5.759962722483092E7 8273 +5.114352166471287E7 8274 +2.768255436087715E7 8275 +6.103292703651686E7 8276 +2.1273498810753115E7 8277 +2.127349890777184E7 8278 +6.103292680403865E7 8279 +4.597354952891349E7 8280 +3.709021575333275E7 8281 +1.2202734587280951E7 8282 +6.103292747529716E7 8283 +9719023.095321089 8284 +5.114352107218414E7 8285 +5.114352275915797E7 8286 +5.114352734519852E7 8287 +2.7682552308759917E7 8288 +4.597354975336351E7 8289 +6.103292700848887E7 8290 +2.768255158272297E7 8291 +9719023.229785616 8292 +6.103294086300542E7 8293 +5.7599627340516165E7 8294 +6.103292692466934E7 8295 +9719023.423502719 8296 +4.5973548806308076E7 8297 +2.768255178153288E7 8298 +2.1273505928189337E7 8299 +5.114352095195711E7 8300 +6.1032929255172975E7 8301 +6.1032927610867985E7 8302 +4.597354927777323E7 8303 +5.11435208165624E7 8304 +5.75996273411996E7 8305 +9719023.10910628 8306 +4.597354924098354E7 8307 +2.127350104688503E7 8308 +5.759962713951002E7 8309 +5.1143520828089595E7 8310 +5.759962812089509E7 8311 +124.48757319420973 8312 +4.59735489305784E7 8313 +1.22027335078821E7 8314 +5.114352289897346E7 8315 +2.7682553728740126E7 8316 +2.7682553032502953E7 8317 +2.768255197662148E7 8318 +6.103292711681729E7 8319 +2.1273498965399902E7 8320 +2.127349834868486E7 8321 +5.759962725069115E7 8322 +5.114352110342437E7 8323 +5.1143520541463874E7 8324 +2.12734984981849E7 8325 +2.7682551626565382E7 8326 +6.1032926971092865E7 8327 +5.7599627041728735E7 8328 +5.7599627674735926E7 8329 +2.1273498567136902E7 8330 +5.1143521651487745E7 8331 +1.2202735232102908E7 8332 +5.759963122959208E7 8333 +2.1273499839115944E7 8334 +5.1143520793964125E7 8335 +5.759963059519656E7 8336 +3.7090212494951844E7 8337 +1.2202734371242307E7 8338 +9719023.662269961 8339 +29.128754024744076 8340 +5.75996271858054E7 8341 +4.5973548939841464E7 8342 +6.103292761326447E7 8343 +5.759962737883318E7 8344 +3.7090221747862846E7 8345 +2.1273499381387502E7 8346 +2.12734989782689E7 8347 +3.70902124940273E7 8348 +1.2202733828946715E7 8349 +3.709021213745661E7 8350 +5.1143520871309206E7 8351 +1.2202733048643421E7 8352 +5.759962720709829E7 8353 +5.759962701997811E7 8354 +5.759963061066384E7 8355 +5.759962729860291E7 8356 +3.709021354405332E7 8357 +4.908551767261165 8358 +5.759962728232207E7 8359 +5.114352117285384E7 8360 +6.103292737785726E7 8361 +5.759962720079659E7 8362 +5.7599627289694585E7 8363 +5.114352066347321E7 8364 +4.5973549349957176E7 8365 +2.1273498665027484E7 8366 +2.7682554899253238E7 8367 +2.768255584779953E7 8368 +1.2202733469908664E7 8369 +2.1273498911454592E7 8370 +2.768255196904537E7 8371 +2.1273499215543002E7 8372 +1.220273380723743E7 8373 +9719023.22025719 8374 +3.709021286970758E7 8375 +1691.955186423738 8376 +2.7682551597243298E7 8377 +8.286006937063691 8378 +5.1143520531632446E7 8379 +5.7599630726819105E7 8380 +3.709021267303784E7 8381 +5.114352165933443E7 8382 +2.1273498316712923E7 8383 +6.103292662839746E7 8384 +2.7682557938922703E7 8385 +2.1273498332471307E7 8386 +4.597354893667192E7 8387 +2.1273498910668366E7 8388 +1.2202734392711021E7 8389 +2.12734985492598E7 8390 +2.7682552064363457E7 8391 +4.217154926749247 8392 +2.768255428633697E7 8393 +5.114352110093866E7 8394 +6.103292855084906E7 8395 +1.2202733923455982E7 8396 +2.7682552935336348E7 8397 +4.597354920027879E7 8398 +5.114352041178699E7 8399 +3.7090212658647604E7 8400 +3.7090212526090056E7 8401 +3.709021246586903E7 8402 +2.768255240949137E7 8403 +3.709021375185731E7 8404 +5.11435223101996E7 8405 +3.709021264031506E7 8406 +2.1273498869795825E7 8407 +2.127349864067689E7 8408 +2.768255225857444E7 8409 +5.11435286498337E7 8410 +5.114352097327853E7 8411 +5.759962714669621E7 8412 +2.127349843304572E7 8413 +4.5973548882626064E7 8414 +5.759962702887675E7 8415 +6.1032928044475794E7 8416 +4.597355007374163E7 8417 +5.1143520795562275E7 8418 +6.103292942301087E7 8419 +2.7682552656428386E7 8420 +4.597354935171283E7 8421 +3.709022858443654E7 8422 +5.1143525148100436E7 8423 +5.759962754871004E7 8424 +1.2202743558237454E7 8425 +5.11435203754971E7 8426 +4.597354944463219E7 8427 +3.032399755473362 8428 +6.1032927241461754E7 8429 +4.597354916843154E7 8430 +3.709021297753147E7 8431 +4.59735488087784E7 8432 +9719023.093624773 8433 +8.80875031982865 8434 +4.5973549381115176E7 8435 +2.768255209989724E7 8436 +5.079483976134905 8437 +6.103292726817049E7 8438 +1.2202733789628489E7 8439 +2.7682560823188327E7 8440 +2.7682551935704604E7 8441 +5.759963247297667E7 8442 +6.10329270863086E7 8443 +3.709021530150362E7 8444 +5.114352604284899E7 8445 +4.597354917591919E7 8446 +22.228694797672357 8447 +1.2202733713489868E7 8448 +9719023.016233489 8449 +9719025.783484014 8450 +2.12734984355431E7 8451 +5.114352086536515E7 8452 +5.759962753763636E7 8453 +5.759963248362962E7 8454 +5.1143520408720665E7 8455 +2.1273499517522734E7 8456 +2.7682601756878994E7 8457 +6.103292665188267E7 8458 +7.253564498628791 8459 +3.709021223093993E7 8460 +1.2202734931317199E7 8461 +4.5973550535284065E7 8462 +4.597354904795913E7 8463 +2.768255199941287E7 8464 +6.103292759581838E7 8465 +2.7682553402643904E7 8466 +2.127350063924772E7 8467 +1.2202733391701631E7 8468 +9719023.190071227 8469 +3.709021523331109E7 8470 +9719023.14787893 8471 +304.3902087443166 8472 +3.709021838026526E7 8473 +9719023.644001922 8474 +1.2202735167260537E7 8475 +5.759962751467032E7 8476 +4.5973549348390065E7 8477 +6.1032926774345145E7 8478 +2.127355890299994E7 8479 +9719023.46390852 8480 +2.7682551957075417E7 8481 +3.7090217721982114E7 8482 +5.1143523020664014E7 8483 +11.208383082813501 8484 +5.759962715323006E7 8485 +9719030.464967612 8486 +2.768255303689679E7 8487 +1.2202775984348161E7 8488 +1.2202733265284145E7 8489 +5.114352152115604E7 8490 +46.80380217750535 8491 +5.1143523108054034E7 8492 +1.220273390313575E7 8493 +1.2202733668528188E7 8494 +9719023.092779003 8495 +3.709021377423583E7 8496 +5.75996271436879E7 8497 +3.709021246130464E7 8498 +4.597354902147471E7 8499 +9719024.735555751 8500 +3.709021371090469E7 8501 +12.733161692438687 8502 +2.127349869031277E7 8503 +2.768255185581162E7 8504 +2.7682552497192867E7 8505 +5.114352106950229E7 8506 +2.7682551571688622E7 8507 +3.709021239121887E7 8508 +6.1032928892704904E7 8509 +1.220273359355172E7 8510 +5.7599627040373385E7 8511 +4.5973549060221836E7 8512 +3.7090212054963104E7 8513 +5.759962735648696E7 8514 +3.709021395569686E7 8515 +4.597354958447106E7 8516 +2.1273498346073836E7 8517 +3.7090213129869886E7 8518 +6.150994083832469 8519 +2.768255229351512E7 8520 +3.709021319131017E7 8521 +5.7599627299507275E7 8522 +3.709021550822948E7 8523 +1.2202733328714857E7 8524 +6.103292775051754E7 8525 +3.709021252393292E7 8526 +9719026.776352087 8527 +4.597355153166741E7 8528 +2.7682566625998925E7 8529 +3.709021710250154E7 8530 +4.5973623012822986E7 8531 +5.759962713483186E7 8532 +3.7090213379326865E7 8533 +2.768255231848664E7 8534 +2.1273499114631534E7 8535 +3.3472504867778796 8536 +4.597354921991587E7 8537 +4.5973548881210916E7 8538 +3.709021220549731E7 8539 +9719023.60282585 8540 +11.459366261908794 8541 +13.782052468658188 8542 +4.597354939007186E7 8543 +1.2202733467315638E7 8544 +2.7682551614293758E7 8545 +5.7599627684451066E7 8546 +5.759963213013866E7 8547 +1.2202733912397597E7 8548 +5.114352090212391E7 8549 +5.114352110720733E7 8550 +5.114352428625403E7 8551 +2.127349842301319E7 8552 +13.145770659848846 8553 +5.1143522561757356E7 8554 +3249.3991570551675 8555 +4.5973549531614706E7 8556 +3.709025054310413E7 8557 +6.103292776283194E7 8558 +1.2202732965129294E7 8559 +6.103292855547624E7 8560 +5.1143521110887945E7 8561 +4.597355041659348E7 8562 +4.597354950900536E7 8563 +2.127349985471208E7 8564 +6.1032927232147224E7 8565 +5.114352182632276E7 8566 +6.103292741312767E7 8567 +2.7682577931241807E7 8568 +4.597354947613848E7 8569 +3.7090213961512096E7 8570 +5.114354662599203E7 8571 +6.1032927417095244E7 8572 +54.033741835998974 8573 +6.1032933080133E7 8574 +9719023.596645568 8575 +2.7682553098833878E7 8576 +9719023.098546803 8577 +3.709021269580818E7 8578 +4.5973550227120176E7 8579 +18.93301095506655 8580 +2.1273498917868514E7 8581 +2.1273503573510103E7 8582 +4.59735489750675E7 8583 +9719026.241446683 8584 +2.7682552239404626E7 8585 +5.114352103624552E7 8586 +2.1273513624024455E7 8587 +5.114352054360975E7 8588 +6.103292710989184E7 8589 +5.759962702387048E7 8590 +6.1032927214668535E7 8591 +6.1032927025228746E7 8592 +3.709021222801487E7 8593 +3.7090213710212834E7 8594 +6.103292920146489E7 8595 +6.1032926633958735E7 8596 +9719023.000321427 8597 +6.1032926855506025E7 8598 +2.12734997218186E7 8599 +17.716305601501674 8600 +3.709021229770848E7 8601 +6.821520075779082 8602 +2.76825520534102E7 8603 +1.2202734079018049E7 8604 +1.220273379274606E7 8605 +2.127349910298158E7 8606 +2.7682553100531813E7 8607 +5.114368243968662E7 8608 +1.2202736440899061E7 8609 +3.709021489127456E7 8610 +2.1273498939621016E7 8611 +1.2202734640978983E7 8612 +1.2202735117625827E7 8613 +2.1273498885571804E7 8614 +1.2202735337305125E7 8615 +9719023.02804925 8616 +3.7090212120070264E7 8617 +5.759962754472182E7 8618 +2.127349842856617E7 8619 +1.2202734399217468E7 8620 +6.103292677596745E7 8621 +9719023.048701186 8622 +11.572766524849145 8623 +6.103292661258243E7 8624 +5.114352041392244E7 8625 +9719023.061532378 8626 +2.1273499189176407E7 8627 +2.127350548015191E7 8628 +1.2202734011066198E7 8629 +5.7599627077017896E7 8630 +5.759962732965509E7 8631 +6.103292717369923E7 8632 +3.709021247570479E7 8633 +2.1273499907670196E7 8634 +5.759962760893708E7 8635 +2.1273499590686724E7 8636 +5.114352364223851E7 8637 +4.597354916523201E7 8638 +6.1032926858938634E7 8639 +6.103292688446599E7 8640 +1.2202733724938653E7 8641 +5.759962710359784E7 8642 +6.103292801818221E7 8643 +9719023.32957075 8644 +3.7090212215803765E7 8645 +1.2202734137430407E7 8646 +5.114352054261792E7 8647 +6.103292985372452E7 8648 +4.597354944180656E7 8649 +5.759962770152629E7 8650 +2.7682552688018262E7 8651 +2.7682552489465926E7 8652 +5.759963123676205E7 8653 +3.709021291186244E7 8654 +2.127349831114773E7 8655 +3.709021465806157E7 8656 +38.85264225953705 8657 +2.768255219489654E7 8658 +4.5973549776405595E7 8659 +9719024.26909799 8660 +5.7599627245468475E7 8661 +6.1032927191441566E7 8662 +9719023.633233787 8663 +9719023.361562055 8664 +23.943712954339407 8665 +6.103293246512265E7 8666 +4.5973548790543824E7 8667 +6.103292650133194E7 8668 +15.288032936511872 8669 +9719023.261929572 8670 +2.127349905374482E7 8671 +2.127350132955483E7 8672 +6.10329283888144E7 8673 +5.759962737351551E7 8674 +1.2202734451782348E7 8675 +3.709021355641026E7 8676 +6.103292652390638E7 8677 +5.114352064008462E7 8678 +9719023.753305227 8679 +2.1273498526281625E7 8680 +9719023.465405665 8681 +9719023.19954778 8682 +1.2202736142612543E7 8683 +1.2202733813338403E7 8684 +9719023.678438308 8685 +3.7090220850066096E7 8686 +4.597354905956216E7 8687 +6.1032926990074866E7 8688 +2.1273499162833508E7 8689 +9719023.236339018 8690 +2.127349950571119E7 8691 +5.759962733002307E7 8692 +1.220273289066274E7 8693 +4.5973552526200905E7 8694 +5.759962720511538E7 8695 +5.114352422564934E7 8696 +1.2202733331762487E7 8697 +6.103292689202562E7 8698 +9719023.141940277 8699 +9719023.771597713 8700 +6.103292717687287E7 8701 +3.709021265507945E7 8702 +3.709021270985085E7 8703 +2.1273498730140805E7 8704 +4.597354932476242E7 8705 +2.127349926409493E7 8706 +2.1273500939654816E7 8707 +4.597355406173867E7 8708 +5.7599627356797636E7 8709 +1.2202734820129897E7 8710 +10.056558800905297 8711 +9719023.227788564 8712 +3.7090212352345794E7 8713 +9719023.205754543 8714 +7.705162482068405 8715 +6.103293528471561E7 8716 +5.759962749932249E7 8717 +3.709021237066967E7 8718 +1.2202733770087779E7 8719 +5.114352084673265E7 8720 +3.709021256504122E7 8721 +2.768255153977893E7 8722 +5.114352108866372E7 8723 +4.597354889880148E7 8724 +3.709021294924806E7 8725 +9719025.026548024 8726 +1.2202734411078108E7 8727 +2.7682552261722103E7 8728 +3.709021236245417E7 8729 +9719023.413756574 8730 +5.759962707923266E7 8731 +6.1032932006906725E7 8732 +9719023.245285442 8733 +3.709021244571109E7 8734 +9719023.235304976 8735 +4.597354982581905E7 8736 +9719040.223430429 8737 +2.76825575767271E7 8738 +20.541287497999097 8739 +21.286269950879984 8740 +9719023.646076273 8741 +6.103292711643897E7 8742 +3.709021583915366E7 8743 +9719023.062889807 8744 +2.7682552961532753E7 8745 +2.7682552301281247E7 8746 +6.1032927283565186E7 8747 +5.46282271869099 8748 +1.2202734293321615E7 8749 +6.103292790574073E7 8750 +5.759962746291308E7 8751 +2.7682551929019272E7 8752 +6.103292861580023E7 8753 +3.4906321079832834 8754 +2.1273498573695898E7 8755 +5.759962703115704E7 8756 +4.597356368786839E7 8757 +7.81083558138291 8758 +1.2202734755680345E7 8759 +7.5216521382008015 8760 +2.7682553924628757E7 8761 +5.759962783396039E7 8762 +17.46770497396257 8763 +2.127349905607775E7 8764 +5.7599627078108676E7 8765 +12.17588579516016 8766 +2.127349991466769E7 8767 +6.103292996342719E7 8768 +4.59735491950412E7 8769 +5.759962731734801E7 8770 +4.479186617382538 8771 +2.1273498710804157E7 8772 +3.709021264810602E7 8773 +3.7090212646828964E7 8774 +3.709021269502703E7 8775 +5.7599628290352486E7 8776 +9719023.025254076 8777 +9719025.021402806 8778 +2.1273498514040474E7 8779 +4.5973549037639216E7 8780 +3.709021903494559E7 8781 +2.1273498571748134E7 8782 +9719025.48126359 8783 +2.1273498630525153E7 8784 +1.2202736394878466E7 8785 +3.709021232429218E7 8786 +4.597354918477746E7 8787 +2.127349888661021E7 8788 +6.1032926729451485E7 8789 +4.597354922804617E7 8790 +4.597354918648846E7 8791 +6.1032937567968085E7 8792 +5.759962733135626E7 8793 +5.759962772168664E7 8794 +2.1273498894511778E7 8795 +2.7682551974777892E7 8796 +1.220273393174974E7 8797 +6.103292975635146E7 8798 +1.220273307935387E7 8799 +1.2202735009359643E7 8800 +3.709021266437202E7 8801 +5.1143520841476984E7 8802 +9719023.51711559 8803 +3.7090212651093066E7 8804 +2.7682553769128855E7 8805 +3.709021213935706E7 8806 +5.759962803748569E7 8807 +1.2202748962270923E7 8808 +9719043.327987991 8809 +5.759962712757875E7 8810 +6.103312209448479E7 8811 +2.1273498461165406E7 8812 +2.1273512136504594E7 8813 +13.837178626631191 8814 +2.1273499575237647E7 8815 +6.103292716277715E7 8816 +10.882497805890665 8817 +1.2202733951573413E7 8818 +1.2202733722814608E7 8819 +5.759965850841333E7 8820 +5.759962900961087E7 8821 +6.105511910238685 8822 +2.1273498903078828E7 8823 +5.114352055281193E7 8824 +9719023.476353701 8825 +5.114352046830626E7 8826 +42.96228830828596 8827 +3.7090212715654425E7 8828 +2.7682552010891803E7 8829 +4.597354930942942E7 8830 +2.768255263496826E7 8831 +1.2202733422982631E7 8832 +6.1032926647562526E7 8833 +2.7682552025422245E7 8834 +6.103293406752597E7 8835 +3.709021431444047E7 8836 +2.7682552576635856E7 8837 +4.597354973218784E7 8838 +5.1143520634543516E7 8839 +2.1273499073740628E7 8840 +2.127350334712705E7 8841 +6.103292694534422E7 8842 +12.30933287142067 8843 +9719023.113936463 8844 +2.7682552066179782E7 8845 +3.70902129731206E7 8846 +6.103292705000798E7 8847 +5.114352073385126E7 8848 +1.2202733769460939E7 8849 +2.1273500546660986E7 8850 +6.103292698129591E7 8851 +2.7682551771656323E7 8852 +6.1032927559342206E7 8853 +2.1273500071333576E7 8854 +5.1143521723906614E7 8855 +9719023.298157528 8856 +1.2202734710490452E7 8857 +6.103292742616433E7 8858 +2.1273499064523976E7 8859 +3.7090213365161E7 8860 +3.7090212125579625E7 8861 +27.405418350081945 8862 +9719023.383352067 8863 +3.709021290153885E7 8864 +63.08125004527199 8865 +9719023.069934484 8866 +5.114352095503916E7 8867 +3.599987022337615 8868 +1.2202733832635397E7 8869 +6.103292704040271E7 8870 +6.103292666761936E7 8871 +2.12734991409963E7 8872 +3.709021269278522E7 8873 +21.272405083970664 8874 +6.103292761942475E7 8875 +2.7682552832433056E7 8876 +5.114352343778003E7 8877 +9719023.977185963 8878 +9719025.031439314 8879 +6.103292684428786E7 8880 +2.768255586747909E7 8881 +9719027.98723247 8882 +2.1273499679923363E7 8883 +1.2202733831003642E7 8884 +4.597355542043468E7 8885 +4.597354916887571E7 8886 +4.597354905628992E7 8887 +2.127349852247444E7 8888 +2.76825614353531E7 8889 +4.597356637940915E7 8890 +9719023.636870608 8891 +1.2202733787123958E7 8892 +4.597354922506587E7 8893 +3.2741024075668324 8894 +1.2202733868885785E7 8895 +7.976175611570562 8896 +9719023.27200423 8897 +1.220273459234195E7 8898 +5.759962739469207E7 8899 +19.392757385705377 8900 +2.768255196903499E7 8901 +4.5973551023522414E7 8902 +1.2202734046977501E7 8903 +4.597354919530861E7 8904 +2.7682552512716588E7 8905 +1.2202732933869211E7 8906 +5.114352114364836E7 8907 +2.76825519600639E7 8908 +1.220277878809467E7 8909 +4.5973550284045E7 8910 +4.59735512373177E7 8911 +1.2202733178068532E7 8912 +2.127349924438653E7 8913 +6.1032927379214585E7 8914 +7.1054790306787075 8915 +5.75996274995818E7 8916 +4.597354905088035E7 8917 +6.103292701797902E7 8918 +5.75996273784745E7 8919 +5.114352140115157E7 8920 +6.1032943689404294E7 8921 +3.709021290420949E7 8922 +4.597355376232414E7 8923 +5.7599627744849235E7 8924 +5.7599627178962894E7 8925 +5.759962704752172E7 8926 +2.7682555553018875E7 8927 +2.127349839337391E7 8928 +6.103292763136328E7 8929 +6.103293127706593E7 8930 +11.0916482271942 8931 +2.7682551543133624E7 8932 +6.1032927722414695E7 8933 +2.7682552356845587E7 8934 +4.59735493490416E7 8935 +9719023.094328282 8936 +6.103293162076939E7 8937 +5.11435243515142E7 8938 +4.5973549429519415E7 8939 +6.1032928802059084E7 8940 +26.549850739788706 8941 +2.76825524122609E7 8942 +3.709021313827869E7 8943 +9719023.185254678 8944 +14.546759361041262 8945 +5.114352059735298E7 8946 +8.762854922857047 8947 +6.1032927527451225E7 8948 +1.2202732879597977E7 8949 +1.2202751436637858E7 8950 +2.768255325986792E7 8951 +9719023.01032558 8952 +6.1032927657407366E7 8953 +2.127350716339958E7 8954 +5.114352184452152E7 8955 +9719023.22315905 8956 +4.59735490506898E7 8957 +4.5973549443481244E7 8958 +5.114352124867792E7 8959 +5.1143521479948536E7 8960 +9719023.580188835 8961 +4.5973548925845414E7 8962 +14.190460673146324 8963 +2.1273498728500377E7 8964 +9719028.419520922 8965 +2.1273498530789815E7 8966 +3.70902120604767E7 8967 +40.44287835204368 8968 +8.670458438595835 8969 +5.666687787145806 8970 +1.2202733561272793E7 8971 +2.1273498928550366E7 8972 +3.709021247059823E7 8973 +4.597354898429176E7 8974 +3.709021284038915E7 8975 +3.70902129472227E7 8976 +4.597354933445089E7 8977 +5.7599627075452715E7 8978 +3.0417772624161987 8979 +2.7682552046741925E7 8980 +1.2202734112554396E7 8981 +4.5973550863268174E7 8982 +9719023.318202185 8983 +2.7682551903844435E7 8984 +5.1143521962437205E7 8985 +3.709021321480453E7 8986 +2.7682556210293073E7 8987 +5.1143520438540116E7 8988 +3.709021468475973E7 8989 +9719024.350996621 8990 +6.103292710813336E7 8991 +10.124640999180993 8992 +1.2202732961157816E7 8993 +4.597354896418889E7 8994 +5.114352054135965E7 8995 +9.759307749367725 8996 +2.127349835845589E7 8997 +9719025.19434284 8998 +2.127349900911523E7 8999 +4.916624764239736 9000 +4.5973549137484126E7 9001 +5.7599627593836136E7 9002 +5.114352294190702E7 9003 +5.75996309325971E7 9004 +2.1273498345136262E7 9005 +51.31001386088724 9006 +1.220273736783047E7 9007 +3.709021310546853E7 9008 +2.127349888756302E7 9009 +3.7090213460995674E7 9010 +3.7090212535827495E7 9011 +5.1143520513022564E7 9012 +4.597354887866835E7 9013 +1.220273347431584E7 9014 +5.7599627135256305E7 9015 +6.163940187925931 9016 +1.2202733002538161E7 9017 +9719023.37495587 9018 +4.597354893433049E7 9019 +1.220273792218396E7 9020 +2.127350052292938E7 9021 +1.2202738205069194E7 9022 +2.1273506575342923E7 9023 +5.114352065245999E7 9024 +3.709021298924117E7 9025 +4.597355006850414E7 9026 +1.220273397652989E7 9027 +10.98197651995884 9028 +4.597354879727837E7 9029 +5.114352072930865E7 9030 +45.88288532981149 9031 +2.7682551608572274E7 9032 +4.513424295618872 9033 +3.709021306881635E7 9034 +5.759962759359081E7 9035 +4.597354939570894E7 9036 +5.759962716175114E7 9037 +2.127349919329494E7 9038 +6.103292695779638E7 9039 +5.759962906753108E7 9040 +1.2202737178624755E7 9041 +9719023.174244655 9042 +9719030.61126318 9043 +1.2202733129309356E7 9044 +6.1032926822644174E7 9045 +26.48645013694785 9046 +2.1273498385871015E7 9047 +3.709022060850606E7 9048 +5.114352064525034E7 9049 +32.396431365544856 9050 +4.597354949432724E7 9051 +1.2202734199461818E7 9052 +2.1273506825836256E7 9053 +6.103292809264308E7 9054 +1.2202734304325495E7 9055 +5.759962733227075E7 9056 +5.759962748909451E7 9057 +5.1143522699373566E7 9058 +6.103292691800573E7 9059 +4.597354899264494E7 9060 +3.709021222090302E7 9061 +2.7682552876096718E7 9062 +3.709021349784453E7 9063 +2.127349896659163E7 9064 +42.5626885813309 9065 +2.127349919699589E7 9066 +1.220273327696918E7 9067 +87.38236136087185 9068 +2.127349895744969E7 9069 +8.187765062483749 9070 +2.1273499200265486E7 9071 +2.1273498883444015E7 9072 +2.127349849995958E7 9073 +5.114352073897135E7 9074 +9719023.28083216 9075 +4.597354963241967E7 9076 +5.114352084561998E7 9077 +4.59735491753585E7 9078 +4.597354898334898E7 9079 +1.220274024940171E7 9080 +9719023.562857503 9081 +4.5973549709596984E7 9082 +5.114352142388652E7 9083 +4.597354942806895E7 9084 +6.103292740553339E7 9085 +4.597354938490363E7 9086 +2.7682552573202785E7 9087 +3.7090215384428635E7 9088 +9719023.093270028 9089 +5.11435209593666E7 9090 +6.103292730552118E7 9091 +9719023.257941566 9092 +2.7682551858387083E7 9093 +4.5973549099145405E7 9094 +2.768255309468592E7 9095 +4.59735489834074E7 9096 +4.304935361227414 9097 +6.103292715863273E7 9098 +5.114352069630223E7 9099 +16.460690348262045 9100 +5.759962742379242E7 9101 +2.127349831694362E7 9102 +2.768255482771474E7 9103 +4.597354976233973E7 9104 +2.768255157106759E7 9105 +5.759963342240302E7 9106 +2.1273498318392698E7 9107 +2.127349905444195E7 9108 +6.1032926740242235E7 9109 +5.114352043420233E7 9110 +9719023.26110574 9111 +5.759962709454411E7 9112 +2.12734993411866E7 9113 +9719023.152408103 9114 +4.5973550412475064E7 9115 +4.5973549284817874E7 9116 +5.1143520582113564E7 9117 +2.1273498589847986E7 9118 +9719023.56652196 9119 +9719023.65531318 9120 +2.7682551536404494E7 9121 +5.759962729335656E7 9122 +5.75996292848125E7 9123 +5.759962764156181E7 9124 +1.220273389869915E7 9125 +1.2202735055469507E7 9126 +5.114352148618407E7 9127 +5.114352119297541E7 9128 +2.7682552680893157E7 9129 +4.59735494727768E7 9130 +5.759962763368903E7 9131 +8.636584548031056 9132 +9719023.25162186 9133 +9719023.021870784 9134 +4.5973548958821304E7 9135 +1.2202733036122534E7 9136 +18.320595710830617 9137 +5.759962751146313E7 9138 +3.7090212861699864E7 9139 +2.7682552473056354E7 9140 +5.759962717639446E7 9141 +9719023.283517174 9142 +50.322456725021304 9143 +6.103292726185271E7 9144 +3.709022600026723E7 9145 +4.5973549676755145E7 9146 +9719023.175357837 9147 +5.114352228883073E7 9148 +1.2202734462950993E7 9149 +3.70902130374327E7 9150 +4.597356338131672E7 9151 +1.2202733781890567E7 9152 +5.7599638217109144E7 9153 +5.75996273586557E7 9154 +1.220273451076405E7 9155 +4.597354900290554E7 9156 +4.597355134482175E7 9157 +2.768255200822599E7 9158 +6.1032926991703175E7 9159 +2.127349915571585E7 9160 +2.1273498317993484E7 9161 +9719023.262239113 9162 +5.114352077877696E7 9163 +4.3036354123022225 9164 +2.1273498646174826E7 9165 +4.597354981950275E7 9166 +5.759962779100797E7 9167 +1.2202735062302131E7 9168 +2.768255227054509E7 9169 +3.709021762443148E7 9170 +9719023.320125198 9171 +5.7599627164051294E7 9172 +4.597354911127765E7 9173 +9719023.3559716 9174 +5.7599627394998975E7 9175 +2.768255165948853E7 9176 +4.5973550827016465E7 9177 +2.768255228867228E7 9178 +2.127349904821226E7 9179 +1.2202733721249552E7 9180 +2.7682552426585425E7 9181 +1.2202741810042001E7 9182 +5.114352120927803E7 9183 +6.103292709911264E7 9184 +5.114352058706219E7 9185 +1.2202734760959864E7 9186 +9719023.5614244 9187 +5.759962841098207E7 9188 +5.1143521125385895E7 9189 +5.114352043963341E7 9190 +2.1273498609447334E7 9191 +2.7682551668429017E7 9192 +4.597357390093191E7 9193 +1.2202734243304102E7 9194 +4.597355002561673E7 9195 +4.5973549117089495E7 9196 +9719024.83855524 9197 +9719023.752202176 9198 +9719023.193458933 9199 +6.1032931808223665E7 9200 +3.7090213010221995E7 9201 +5.7599627529029995E7 9202 +9719023.07240841 9203 +9719023.9002044 9204 +9719023.562509246 9205 +4.597356793314584E7 9206 +9719023.807339445 9207 +3.7090215791050635E7 9208 +2.1273500911425512E7 9209 +1.2202734322362112E7 9210 +3.709021632330318E7 9211 +3.709021424373919E7 9212 +9719023.082442263 9213 +4.597354935225021E7 9214 +2.1273499093068216E7 9215 +5.7599627246978186E7 9216 +9719023.045421239 9217 +1.2202733120400742E7 9218 +4.597354894100314E7 9219 +4.5973548789070874E7 9220 +9719023.249668539 9221 +3.7090212322033E7 9222 +22.17247904949819 9223 +2.768255445215971E7 9224 +2.768255645445979E7 9225 +5.055272155361124 9226 +1.2202733272513827E7 9227 +4.597354903572262E7 9228 +1.2202733498434851E7 9229 +9719023.166254342 9230 +5.759962756544079E7 9231 +2.1273498580671854E7 9232 +4.8589406725444295 9233 +1.2202733468879757E7 9234 +6.1032927161196224E7 9235 +3.709021349384206E7 9236 +9719023.461369656 9237 +1.2202733231213935E7 9238 +2.7682553597630456E7 9239 +4.597354924306597E7 9240 +9719023.59514894 9241 +5.7599627503812775E7 9242 +3.7090212474499725E7 9243 +5.261637309193155 9244 +2.127349905270716E7 9245 +9719024.915364357 9246 +9719024.31385745 9247 +37.39819051962221 9248 +20.384415703622597 9249 +4.597355135393945E7 9250 +5.759962726173982E7 9251 +5.1143521242584184E7 9252 +9719023.584041266 9253 +4.5973549422714256E7 9254 +1.220273323115033E7 9255 +1.2202733343388628E7 9256 +1.2202733127942806E7 9257 +5.759962776167342E7 9258 +2.127349881828459E7 9259 +1.220273291836526E7 9260 +5.1143520525171265E7 9261 +4.597354921005909E7 9262 +2.7682552414479267E7 9263 +5.114352406283394E7 9264 +6.103294396511179E7 9265 +1.2202734143108988E7 9266 +2.7682554775904745E7 9267 +4.554147186160563 9268 +5.759962724868261E7 9269 +2.768255316425345E7 9270 +3.9197812861573755 9271 +2.7682551636204477E7 9272 +1.2202733893263472E7 9273 +3.709021320687914E7 9274 +3.709021216332665E7 9275 +55.374729476918574 9276 +2.1273498365314864E7 9277 +6.1032946975831635E7 9278 +6.103292651542448E7 9279 +3.709021238749068E7 9280 +9719023.151271736 9281 +9719023.686380792 9282 +5.114352250399227E7 9283 +1.2202734028070185E7 9284 +5.759962720475778E7 9285 +5.114352420722346E7 9286 +20.41248360040555 9287 +5.11435278895795E7 9288 +5.759962712738818E7 9289 +2.1273498875028197E7 9290 +4.993180547508204 9291 +1.2202733357974624E7 9292 +1.2202738973119494E7 9293 +1.22027332702075E7 9294 +2.7682553801329803E7 9295 +2.1273498835432857E7 9296 +2.1273498677982572E7 9297 +4.597354897199781E7 9298 +6.499197628528193 9299 +9719023.243788404 9300 +2.127349955959119E7 9301 +3.709021215111853E7 9302 +9719023.102399109 9303 +6.1032932773719296E7 9304 +5.7599627052594624E7 9305 +17.58248223365404 9306 +1.2202733836576164E7 9307 +5.1143521567297265E7 9308 +4.387826106395104 9309 +4.597354927646773E7 9310 +6.103292671966195E7 9311 +2.7682552337115318E7 9312 +5.114352196444436E7 9313 +2.7682551813364282E7 9314 +3.709021206124715E7 9315 +2.7682551783112776E7 9316 +1.2202733492720678E7 9317 +9719023.014338698 9318 +3.709021575391097E7 9319 +1.22028079152324E7 9320 +4.597354902103904E7 9321 +2.12734996234637E7 9322 +5.7599628525969915E7 9323 +5.7599627159034975E7 9324 +5.114352086988471E7 9325 +1.220273325209093E7 9326 +9719023.17894091 9327 +6.1032927557221636E7 9328 +5.1143520678943984E7 9329 +2.7682552968271267E7 9330 +2.12734985368005E7 9331 +4.5973549430761024E7 9332 +25.031149648758095 9333 +5.759962708406028E7 9334 +2.768255270237305E7 9335 +6.1032926627139166E7 9336 +3.709021280835054E7 9337 +4.597354879152195E7 9338 +2.1273499187245335E7 9339 +5.114352202815164E7 9340 +9719023.478338178 9341 +3.709021225100155E7 9342 +6.103292755775703E7 9343 +4.5973549561591946E7 9344 +3.709021408019356E7 9345 +2.1273498461039957E7 9346 +3.709021295325277E7 9347 +10.8495120092635 9348 +5.7599627107381545E7 9349 +2.7682551813741352E7 9350 +2.7682552486425266E7 9351 +19.050176252117534 9352 +9719023.436412742 9353 +5.759962766379325E7 9354 +5.114352085754317E7 9355 +2.768255158263303E7 9356 +3.709021213272884E7 9357 +5.1143521053040355E7 9358 +6.1032927750435024E7 9359 +3.70902137840592E7 9360 +6.103293145488943E7 9361 +4.5973549371469095E7 9362 +6.1032926548597544E7 9363 +9719023.527609836 9364 +1.220274386181149E7 9365 +3.7090212333282836E7 9366 +5.759962713265624E7 9367 +5.114352091134592E7 9368 +5.1143520896424845E7 9369 +3.7090214371519506E7 9370 +2.768255395357963E7 9371 +6.103292710325371E7 9372 +4.5973548791199364E7 9373 +5.7599627394999966E7 9374 +4.5973548994502306E7 9375 +2.127349849046047E7 9376 +1.2202734409535669E7 9377 +9719023.10354793 9378 +6.103292699443453E7 9379 +2.7682637856990717E7 9380 +5.11435219662997E7 9381 +1.220273467977278E7 9382 +6.103292775941053E7 9383 +2.768255213150569E7 9384 +2.7682551881955706E7 9385 +3.709021363724981E7 9386 +2.1273499451309472E7 9387 +5.75996270824898E7 9388 +9719023.519565688 9389 +6.1032927949126735E7 9390 +6.103293892081475E7 9391 +2.768255445040616E7 9392 +2.76825524391675E7 9393 +3.709021269693545E7 9394 +2.7682551598175693E7 9395 +9719023.114380443 9396 +5.114352100729859E7 9397 +4.5973552314904764E7 9398 +9719023.418880478 9399 +5.1143520369191945E7 9400 +3.709021311453801E7 9401 +5.759963076921708E7 9402 +5.759962737681582E7 9403 +3.7090213169602506E7 9404 +9719023.120885085 9405 +9719023.0277483 9406 +4.597354953893252E7 9407 +5.7599627158207186E7 9408 +9719023.096706044 9409 +5.7599627099922076E7 9410 +2.1273506858553305E7 9411 +5.224392420005926 9412 +6.103292654543392E7 9413 +5.1143520919371925E7 9414 +4.5973549799383335E7 9415 +9719023.048529731 9416 +4.5973548828042954E7 9417 +2.127349895432185E7 9418 +1.2202733822503569E7 9419 +9719024.095401855 9420 +5.759962719307854E7 9421 +2.1273498479539864E7 9422 +1.2202733392257333E7 9423 +9719023.078018151 9424 +6.1032934679485746E7 9425 +5.759962707759452E7 9426 +1.220273362046864E7 9427 +5.114352280880371E7 9428 +4.5973549119740024E7 9429 +25.243996374555728 9430 +4.59735489067778E7 9431 +5.7599627081790335E7 9432 +4.597354997685713E7 9433 +5.759962819544497E7 9434 +18.588481829443968 9435 +2.1273499251988493E7 9436 +4.597354921863112E7 9437 +2.127349923985207E7 9438 +5.759962713286138E7 9439 +2.7682552498202078E7 9440 +9719023.53864265 9441 +4.597354936118308E7 9442 +5.75996274951409E7 9443 +6.680097157229583 9444 +2.7682551615351263E7 9445 +5.759962719074242E7 9446 +2.1273498751876127E7 9447 +5.759962735550375E7 9448 +1.2202742909009717E7 9449 +6.182047572163187 9450 +2.12735462150336E7 9451 +3.709021210343586E7 9452 +2.7682551589624304E7 9453 +2.127349849550906E7 9454 +4.59735493048663E7 9455 +15.856995807276629 9456 +6.1032929509281136E7 9457 +4.597354979006672E7 9458 +2.768255204114487E7 9459 +5.759962757423119E7 9460 +3.691684776017655 9461 +3.709021214739019E7 9462 +6.103292670830754E7 9463 +5.114352153400892E7 9464 +2.7682552850602873E7 9465 +2.127349940188891E7 9466 +5.1143520592901126E7 9467 +5.114352133771559E7 9468 +4.59735495295623E7 9469 +9719023.092909519 9470 +2.768260452073811E7 9471 +9719023.239367742 9472 +9719023.44610524 9473 +2.127349870905171E7 9474 +2.127349910336306E7 9475 +2.1273500863847733E7 9476 +5.759962838142046E7 9477 +3.709021231268647E7 9478 +9719023.283532841 9479 +5.772728210033363 9480 +5.1143521001499526E7 9481 +9719039.075743357 9482 +1.220273395091676E7 9483 +4.597355015436951E7 9484 +9719025.494836528 9485 +15.532109485862936 9486 +2.1273498455098525E7 9487 +4.597354919718394E7 9488 +5.759962713040794E7 9489 +1.220273307870207E7 9490 +5.759962733817605E7 9491 +1.2202732954076963E7 9492 +6.103292733940251E7 9493 +5.114352062778048E7 9494 +1.2202734416361375E7 9495 +5.1143520907253444E7 9496 +6.103292663441711E7 9497 +6.103292700611603E7 9498 +9719023.085220223 9499 +5.759962709164459E7 9500 +1.2202733362460945E7 9501 +6.103292658427867E7 9502 +2.1273498399877038E7 9503 +4.5973548940658465E7 9504 +4.597356403961976E7 9505 +6.103292682109795E7 9506 +6.103292792310834E7 9507 +6.103292897430335E7 9508 +2.768255431049759E7 9509 +2.7682553503285863E7 9510 +1.2202733803169755E7 9511 +22.515359171121236 9512 +13.922835799326068 9513 +1.2202733487864222E7 9514 +2.7682552634793404E7 9515 +2.12734983259571E7 9516 +196.07734839054265 9517 +1.220273378219972E7 9518 +1.2202733557617327E7 9519 +5.11435226523604E7 9520 +5.7599627830595285E7 9521 +19.941845036576286 9522 +1.2202735602940999E7 9523 +3.709021225114762E7 9524 +4.597354894559855E7 9525 +9719034.250061342 9526 +6.103292720166856E7 9527 +5.759962823339161E7 9528 +23.11733376260809 9529 +9719023.351864347 9530 +2.1273498754555438E7 9531 +5.759962743557589E7 9532 +2.7682551606978364E7 9533 +18.331509826282847 9534 +2.76825623138526E7 9535 +5.75996271661695E7 9536 +1.2202733497347958E7 9537 +4.597354910289838E7 9538 +5.114352042196129E7 9539 +3.7090212915090226E7 9540 +1.2202736845314134E7 9541 +3.709021267179035E7 9542 +2.1273498842348382E7 9543 +4.597355026016836E7 9544 +5.342617129402909 9545 +2.7682552016431212E7 9546 +2.127349874003665E7 9547 +2.127349850673897E7 9548 +8.432322626638104 9549 +9719023.071024047 9550 +5.7599627101450145E7 9551 +2.1273499401975453E7 9552 +2.7682552138330985E7 9553 +16.92133006040558 9554 +6.103293324845937E7 9555 +2.1273498498716243E7 9556 +2.768255298894542E7 9557 +9719023.20317582 9558 +2.1273499191984598E7 9559 +4.597354970819128E7 9560 +5.114352068613814E7 9561 +5.028326120403319 9562 +2.127551442830098E7 9563 +9719023.061309602 9564 +4.597354944054191E7 9565 +3.7090213021988995E7 9566 +6.103292682475325E7 9567 +9719023.17688686 9568 +33.31526761966617 9569 +5.759962760654732E7 9570 +5.114352088989838E7 9571 +1.220290175451458E7 9572 +5.7599650663134545E7 9573 +2.1273498947363798E7 9574 +9719023.141588682 9575 +52.12080444211868 9576 +9719023.02895344 9577 +5.759962739375601E7 9578 +6.1032927430902444E7 9579 +5.75996275370298E7 9580 +2.1273498955329947E7 9581 +5.7599628340845264E7 9582 +2.1273498431799814E7 9583 +3.284366905598281 9584 +2.127349840315242E7 9585 +9719023.078386948 9586 +5.1143520908455096E7 9587 +5.457845932425275 9588 +3.709021543780256E7 9589 +3.709021414293791E7 9590 +5.759962734977974E7 9591 +5.759962748671661E7 9592 +5.759962763228085E7 9593 +2.1273498941121075E7 9594 +3.7090213525646664E7 9595 +6.103292788260319E7 9596 +9719023.199201234 9597 +9719023.42574984 9598 +5.759962732171067E7 9599 +6.103292730227887E7 9600 +5.1143521099847995E7 9601 +5.759962725586276E7 9602 +2.7682565435124107E7 9603 +5.11435277061993E7 9604 +5.759962703001926E7 9605 +5.759962737986019E7 9606 +4.5973549422766216E7 9607 +6.103292759173498E7 9608 +1.2202737017777335E7 9609 +6.103294043588144E7 9610 +5.114352100928907E7 9611 +4.5973548915436424E7 9612 +2.127350033663759E7 9613 +1.220275413912304E7 9614 +6.103293567941509E7 9615 +5.1143521622623816E7 9616 +6.1032934765568E7 9617 +31.20282608950115 9618 +4.29940319848356 9619 +6.103292957130987E7 9620 +9719023.040882805 9621 +5.759962707130048E7 9622 +3.709021293548886E7 9623 +1.2202733017718937E7 9624 +1.2202733782700045E7 9625 +2.127349871360132E7 9626 +6.103292689469701E7 9627 +1.2202733946898952E7 9628 +6.103292741863138E7 9629 +2.7682553220091663E7 9630 +4.597354896585571E7 9631 +2.1273498843235675E7 9632 +3.7090212137380615E7 9633 +10.363642168488097 9634 +5.114352112995913E7 9635 +3.70902129339905E7 9636 +5.759962734231014E7 9637 +2.127349941941174E7 9638 +4.531716517339298 9639 +5.759962715425779E7 9640 +5.759962729407936E7 9641 +2.768255250906244E7 9642 +5.114352206793712E7 9643 +8.982590227061326 9644 +6.1032926545897044E7 9645 +5.759962742595197E7 9646 +9719023.36773432 9647 +2.127349871083425E7 9648 +6.103292700665009E7 9649 +4.306661179658711 9650 +5.759962874001687E7 9651 +1.220273925604195E7 9652 +4.5973552485423476E7 9653 +3.70902139672728E7 9654 +5.759962716410684E7 9655 +2.1273498462057903E7 9656 +6.1032926821518354E7 9657 +2.7682551758584224E7 9658 +5.7599630090955816E7 9659 +2.1273498680489145E7 9660 +5.759962728179097E7 9661 +2.127349873824993E7 9662 +9719023.609726667 9663 +2.7682556457081456E7 9664 +2.1273501119528946E7 9665 +1.2202733771655243E7 9666 +6.1032927470728636E7 9667 +4.597354978886482E7 9668 +5.1143520826504275E7 9669 +6.3404630503935575 9670 +4.597354883942227E7 9671 +5.759962705955652E7 9672 +5.759962731893577E7 9673 +4.597354920282909E7 9674 +2.768255228124121E7 9675 +6.1032928732294045E7 9676 +9719055.053285526 9677 +2.1273498885328945E7 9678 +4.5973550817471124E7 9679 +4.5973549320359014E7 9680 +8.575766319216388 9681 +5.75996274672471E7 9682 +5.114352045025625E7 9683 +4.5973548980339006E7 9684 +11.291378017126478 9685 +3.4259911080401046 9686 +6.103292697434798E7 9687 +2.7682551898611713E7 9688 +4.597354950465496E7 9689 +5.7599627236387186E7 9690 +5.1143520477922015E7 9691 +2.7682558257651806E7 9692 +1.2202732906102387E7 9693 +2.1273531188259885E7 9694 +4.59735494248653E7 9695 +4.597354941484536E7 9696 +5.1143521030356966E7 9697 +5.114352107560713E7 9698 +2.1273500756005444E7 9699 +9719023.3184824 9700 +6.103292686685418E7 9701 +1.220273399833934E7 9702 +5.7599627226861775E7 9703 +2.7682552444178157E7 9704 +6.1032928913924865E7 9705 +2.7682552370016236E7 9706 +1.220273359629046E7 9707 +2.7682552014655273E7 9708 +4.961562913034459 9709 +5.11435209426289E7 9710 +3.709021205726725E7 9711 +5.114352599446866E7 9712 +6.1032926585587904E7 9713 +4.597354902840905E7 9714 +2.127349894317715E7 9715 +5.104686238264837 9716 +5.114352097437545E7 9717 +2.7682559033540595E7 9718 +3.7090212494251035E7 9719 +2.7682552104780387E7 9720 +3.709021785517543E7 9721 +4.207956755977165 9722 +2.1273498594368316E7 9723 +1.2202734547668906E7 9724 +2.1273499688685972E7 9725 +9719023.666544048 9726 +5.114352062124043E7 9727 +5.114352046748507E7 9728 +1.2202734511018176E7 9729 +5.7599627985204905E7 9730 +5.114352089914174E7 9731 +9719023.191823795 9732 +3.5802620016334337 9733 +5.114352042525757E7 9734 +31.270709703803522 9735 +2.7682553140645478E7 9736 +4.597355017398196E7 9737 +9719023.197923679 9738 +5.1143520707511686E7 9739 +73.83260898191827 9740 +1.2202733098617185E7 9741 +2.1273498409412034E7 9742 +3.709021451058991E7 9743 +5.114352172940424E7 9744 +9719024.139131708 9745 +3.70902145779736E7 9746 +2.7682554327430844E7 9747 +5.759962711107302E7 9748 +2.127349892794281E7 9749 +6.103292781900284E7 9750 +10.658639403842837 9751 +177.27398481346347 9752 +3.709021243595702E7 9753 +1.2202733391131364E7 9754 +10.56167614214526 9755 +1.2202734036208667E7 9756 +1.2202733999818249E7 9757 +1.220282624619902E7 9758 +1.2202733958248122E7 9759 +3.7090212605840616E7 9760 +1.2202733797377605E7 9761 +4.59735490660943E7 9762 +6.103292993472197E7 9763 +15.712660404841678 9764 +5.11435210643037E7 9765 +3.709022345148001E7 9766 +2.768255231280019E7 9767 +3.709021254225355E7 9768 +2.768255174939039E7 9769 +187.02489481693215 9770 +5.7599634173676975E7 9771 +5.7599627282579705E7 9772 +5.1143521024301104E7 9773 +2.1273498915567238E7 9774 +5.114352154684984E7 9775 +1.2202738116752982E7 9776 +6.103292687496904E7 9777 +4.5973551409419574E7 9778 +5.1143520534631275E7 9779 +1.2202733807722125E7 9780 +3.6554232871266716 9781 +2.7682551889629867E7 9782 +6.1032927066652544E7 9783 +4.5973549080536544E7 9784 +67.99752871427242 9785 +5.1143521034978546E7 9786 +4.5973551655123234E7 9787 +8.022919175430529 9788 +1.220273347324418E7 9789 +2.127349893239656E7 9790 +2.7682551537052244E7 9791 +2.1273499011650283E7 9792 +5.114352045620856E7 9793 +6.10329268297997E7 9794 +71.14397837400224 9795 +6.36458073031213 9796 +1.2202733573001798E7 9797 +5.114352045592186E7 9798 +4.597354930794885E7 9799 +6.1032927800294645E7 9800 +5.759962981132145E7 9801 +4.5973549190520436E7 9802 +9719023.16269416 9803 +2.1273501633313283E7 9804 +5.75996386076551E7 9805 +3.709021341841239E7 9806 +6.103292708261587E7 9807 +2.1273498504641593E7 9808 +2.1273498562109087E7 9809 +5.114352075765701E7 9810 +2.1273502034159783E7 9811 +3.709021246943951E7 9812 +5.759962707159749E7 9813 +13.501812121217226 9814 +41.95872953044737 9815 +1.2202734168740919E7 9816 +5.114352516944604E7 9817 +6.103293239255476E7 9818 +4.597354886388235E7 9819 +5.75996275996155E7 9820 +5.759962718608822E7 9821 +3.7090212578294374E7 9822 +6954.759552365898 9823 +3.7090212380078144E7 9824 +4.597354920405236E7 9825 +5.7599653742179796E7 9826 +6.1032926642698325E7 9827 +3.7090213096150875E7 9828 +5.759962724817129E7 9829 +6.103292691310494E7 9830 +2.127350137406097E7 9831 +3.709021314793893E7 9832 +3.709021211387315E7 9833 +22.23827135745945 9834 +2.1273505848420307E7 9835 +58.00655056429222 9836 +2.7682552702916622E7 9837 +3.7090212559150085E7 9838 +2.768255158724024E7 9839 +5.114352123008716E7 9840 +3.709021658206879E7 9841 +4.597355142409255E7 9842 +4.597355164508029E7 9843 +1.2202733763515059E7 9844 +5.759962719233209E7 9845 +5.1143522035285346E7 9846 +4.597354900216359E7 9847 +9719024.534525216 9848 +5.759962743723247E7 9849 +2.768255258195342E7 9850 +4.597354898163566E7 9851 +2.768255177202573E7 9852 +2.768255223585632E7 9853 +5.759962713918523E7 9854 +5.1143522302481584E7 9855 +2.1273498671635665E7 9856 +6.103292674724871E7 9857 +1.2202733428724255E7 9858 +1.2202732948476037E7 9859 +5.759962761231611E7 9860 +9719023.265079819 9861 +4.597354987367344E7 9862 +11.559357752773145 9863 +5.114352355356348E7 9864 +2.1273499266133137E7 9865 +56.204391138572895 9866 +4.5973548857962355E7 9867 +9719023.168979893 9868 +9719023.546028776 9869 +1.2202733953987801E7 9870 +5.7599627167104766E7 9871 +5.7599627218502894E7 9872 +5.1143521102951705E7 9873 +5.114352224921535E7 9874 +5.1143521077046335E7 9875 +3.7090212097582564E7 9876 +5.114352047481036E7 9877 +6.103292664993121E7 9878 +3.709021288418004E7 9879 +6.103292718243246E7 9880 +11.315169030239336 9881 +2.7682553715489727E7 9882 +27.416347672439667 9883 +5.1143521033660024E7 9884 +4.5973549019383594E7 9885 +9719024.21898817 9886 +5.11435203977515E7 9887 +5.759962730275058E7 9888 +3.709022267947103E7 9889 +9719023.097447177 9890 +2.7682552783709276E7 9891 +6.1032927471838646E7 9892 +2.7682551755524E7 9893 +2.7682552185430385E7 9894 +2.1273499847551834E7 9895 +4.5973550407668665E7 9896 +6.29663170152677 9897 +10.959467122090063 9898 +5.7599627947517246E7 9899 +5.11435203925176E7 9900 +4.5973550099713095E7 9901 +3.709021223519461E7 9902 +4.597355061633791E7 9903 +5.759962728171773E7 9904 +5.759963840353744E7 9905 +31.360889655031542 9906 +5.7599632083762E7 9907 +3.7090212440990105E7 9908 +5.114352058930146E7 9909 +9719023.318459615 9910 +9719023.49415912 9911 +6.10329286975474E7 9912 +5.1143523708834805E7 9913 +9719023.60753108 9914 +1.2203593129791355E7 9915 +5.114352095413233E7 9916 +1.2202733365732133E7 9917 +5.114352067681469E7 9918 +11.760606018200608 9919 +4.5973550552343965E7 9920 +6.1032926783732854E7 9921 +2.127349871347649E7 9922 +2.1273498586561907E7 9923 +1.2202733065799788E7 9924 +30.451827119128794 9925 +9719023.420641683 9926 +9719023.115947064 9927 +5.7599627241424255E7 9928 +5.759962740579218E7 9929 +5.114352523554401E7 9930 +14.377304237129911 9931 +9719031.858477307 9932 +5.114353996782986E7 9933 +9719023.055727864 9934 +160.83260338279894 9935 +5.759962735293234E7 9936 +1.2202734240350198E7 9937 +1.2202733858671388E7 9938 +2.7682552196066055E7 9939 +5.759962729064731E7 9940 +9719023.100336567 9941 +1.22027329080031E7 9942 +5.7599627459016435E7 9943 +2.768255918684231E7 9944 +5.114352165386036E7 9945 +9719024.785901899 9946 +3.709021249247179E7 9947 +3.709021252910861E7 9948 +5.1143521064899884E7 9949 +3.7090213417875916E7 9950 +2.768255163962757E7 9951 +5.7599627468497075E7 9952 +6.103292748305372E7 9953 +4.597354937718751E7 9954 +7.25511889627341 9955 +5.1143520954479314E7 9956 +2.76825521616908E7 9957 +4.597354922125976E7 9958 +53.03109147090821 9959 +5.759962735899415E7 9960 +3.7090212263672695E7 9961 +2.7682553596577134E7 9962 +5.114352048450694E7 9963 +4.59735490068617E7 9964 +2.1273500154558063E7 9965 +5.759962734412891E7 9966 +1.2202733043209473E7 9967 +5.1143521076555885E7 9968 +1.2202734062225234E7 9969 +4.5973549344710566E7 9970 +18.11015157550133 9971 +2.7682551869138613E7 9972 +6.1032930836168915E7 9973 +6.325999586403711 9974 +9719023.440162867 9975 +5.114352094893948E7 9976 +5.1143520621365495E7 9977 +4.832785263718778 9978 +4.5973549251923814E7 9979 +6.103292650807905E7 9980 +3.7090212928294554E7 9981 +3.709021561415685E7 9982 +5.11435206731743E7 9983 +6.103292685379528E7 9984 +5.7599627370324865E7 9985 +5.114352068278065E7 9986 +15.454753262345696 9987 +6.103292764539063E7 9988 +5.759962731333177E7 9989 +1.2202739466566179E7 9990 +36.63140893298423 9991 +4.597354956979832E7 9992 +5.7599634525777414E7 9993 +5.114352054752105E7 9994 +3.7090212336321555E7 9995 +5.114352118660649E7 9996 +6.103292832038828E7 9997 +2.1273498890264012E7 9998 +2.127349874189416E7 9999 diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/ConnectorPolicyAssignmentPolicy.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/ConnectorPolicyAssignmentPolicy.java new file mode 100644 index 00000000000..4654ed78fb8 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/ConnectorPolicyAssignmentPolicy.java @@ -0,0 +1,42 @@ +/* + * Copyright 2009-2013 by The Regents of the University of California + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * you may obtain a copy of the License from + * + * 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.hyracks.tests.integration; + +import org.apache.hyracks.api.dataflow.IConnectorDescriptor; +import org.apache.hyracks.api.dataflow.connectors.IConnectorPolicy; +import org.apache.hyracks.api.dataflow.connectors.IConnectorPolicyAssignmentPolicy; +import org.apache.hyracks.api.dataflow.connectors.PipeliningConnectorPolicy; +import org.apache.hyracks.api.dataflow.connectors.SendSideMaterializedPipeliningConnectorPolicy; +import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor; + +/** + * @author michael + */ + +public class ConnectorPolicyAssignmentPolicy implements IConnectorPolicyAssignmentPolicy { + private static final long serialVersionUID = 1L; + private IConnectorPolicy senderSideMaterializePolicy = new SendSideMaterializedPipeliningConnectorPolicy(); + private IConnectorPolicy pipeliningPolicy = new PipeliningConnectorPolicy(); + + @Override + public IConnectorPolicy getConnectorPolicyAssignment(IConnectorDescriptor c, int nProducers, int nConsumers, + int[] fanouts) { + if (c instanceof MToNPartitioningMergingConnectorDescriptor) { + return senderSideMaterializePolicy; + } else { + return pipeliningPolicy; + } + } +} diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/SampleBandJoinTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/SampleBandJoinTest.java new file mode 100644 index 00000000000..481f83ea7e7 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/SampleBandJoinTest.java @@ -0,0 +1,119 @@ +/* + * Copyright 2009-2013 by The Regents of the University of California + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * you may obtain a copy of the License from + * + * 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.hyracks.tests.integration; + +import java.io.File; + +import org.junit.Test; + +import org.apache.hyracks.api.constraints.PartitionConstraintHelper; +import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory; +import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer; +import org.apache.hyracks.api.dataflow.value.RecordDescriptor; +import org.apache.hyracks.api.io.FileReference; +import org.apache.hyracks.api.job.JobSpecification; +import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory; +import org.apache.hyracks.data.std.primitive.UTF8StringPointable; +import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer; +import org.apache.hyracks.dataflow.common.data.parsers.IValueParserFactory; +import org.apache.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory; +import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor; +import org.apache.hyracks.dataflow.std.file.ConstantFileSplitProvider; +import org.apache.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory; +import org.apache.hyracks.dataflow.std.file.FileScanOperatorDescriptor; +import org.apache.hyracks.dataflow.std.file.FileSplit; +import org.apache.hyracks.dataflow.std.file.IFileSplitProvider; +import org.apache.hyracks.dataflow.std.misc.MaterializingOperatorDescriptor; +import org.apache.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor; + +/** + * @author michael + */ +public class SampleBandJoinTest extends AbstractIntegrationTest { + private static final boolean DEBUG = false; + + @Test + public void sampleForward_Case1() throws Exception { + JobSpecification spec = new JobSpecification(); + FileSplit[] custSplits = new FileSplit[] { + new FileSplit(NC1_ID, new FileReference(new File("data/tpch0.001/customer-part1.tbl"))), + new FileSplit(NC2_ID, new FileReference(new File("data/tpch0.001/customer-part2.tbl"))) }; + IFileSplitProvider custSplitsProvider = new ConstantFileSplitProvider(custSplits); + RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] { + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer() }); + + FileSplit[] ordersSplits = new FileSplit[] { + new FileSplit(NC1_ID, new FileReference(new File("data/tpch0.001/orders-part1.tbl"))), + new FileSplit(NC2_ID, new FileReference(new File("data/tpch0.001/orders-part2.tbl"))) }; + + IFileSplitProvider ordersSplitsProvider = new ConstantFileSplitProvider(ordersSplits); + RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] { + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer() }); + + RecordDescriptor custOrderJoinDesc = new RecordDescriptor(new ISerializerDeserializer[] { + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer() }); + + FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitsProvider, + new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE, + UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, + UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, + UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, + UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID, NC2_ID); + + FileScanOperatorDescriptor custScanner = new FileScanOperatorDescriptor(spec, custSplitsProvider, + new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE, + UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, + UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, + UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, + UTF8StringParserFactory.INSTANCE }, '|'), custDesc); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID); + + ExternalSortOperatorDescriptor sorterOrd = new ExternalSortOperatorDescriptor(spec, 4, new int[] { 1 }, + new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), + PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }, ordersDesc); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorterOrd, NC1_ID, NC2_ID); + spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorterOrd, 0); + + ExternalSortOperatorDescriptor sorterCust = new ExternalSortOperatorDescriptor(spec, 4, new int[] { 0 }, + new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), + PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }, custDesc); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorterCust, NC1_ID, NC2_ID); + spec.connect(new OneToOneConnectorDescriptor(spec), custScanner, 0, sorterCust, 0); + + MaterializingOperatorDescriptor materOrd = new MaterializingOperatorDescriptor(spec, ordersDesc); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, materOrd, NC1_ID, NC2_ID); + spec.connect(new OneToOneConnectorDescriptor(spec), sorterOrd, 0, materOrd, 0); + + MaterializingOperatorDescriptor materCust = new MaterializingOperatorDescriptor(spec, custDesc); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, materCust, NC1_ID, NC2_ID); + spec.connect(new OneToOneConnectorDescriptor(spec), sorterCust, 0, materCust, 0); + } +} diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/SampleForwardTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/SampleForwardTest.java new file mode 100644 index 00000000000..57bcc4d6575 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/SampleForwardTest.java @@ -0,0 +1,664 @@ +/* + * Copyright 2009-2013 by The Regents of the University of California + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * you may obtain a copy of the License from + * + * 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.hyracks.tests.integration; + +import java.io.File; +import java.nio.ByteBuffer; + +import org.junit.Test; +import org.apache.hyracks.api.constraints.PartitionConstraintHelper; +import org.apache.hyracks.api.dataflow.IConnectorDescriptor; +import org.apache.hyracks.api.dataflow.IOperatorDescriptor; +import org.apache.hyracks.api.dataflow.connectors.ConnectorPolicyFactory; +import org.apache.hyracks.api.dataflow.connectors.IConnectorPolicy; +import org.apache.hyracks.api.dataflow.connectors.IConnectorPolicyAssignmentPolicy; +import org.apache.hyracks.api.dataflow.connectors.PipeliningConnectorPolicy; +import org.apache.hyracks.api.dataflow.connectors.SendSideMaterializedBlockingConnectorPolicy; +import org.apache.hyracks.api.dataflow.connectors.SendSideMaterializedPipeliningConnectorPolicy; +import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory; +import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFactory; +import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory; +import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer; +import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory; +import org.apache.hyracks.api.dataflow.value.RecordDescriptor; +import org.apache.hyracks.api.dataset.ResultSetId; +import org.apache.hyracks.api.io.FileReference; +import org.apache.hyracks.api.job.JobSpecification; +import org.apache.hyracks.data.std.accessors.PointableBinaryComparatorFactory; +import org.apache.hyracks.data.std.accessors.PointableBinaryHashFunctionFactory; +import org.apache.hyracks.data.std.primitive.IntegerPointable; +import org.apache.hyracks.data.std.primitive.UTF8StringPointable; +import org.apache.hyracks.dataflow.common.data.marshalling.FloatSerializerDeserializer; +import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer; +import org.apache.hyracks.dataflow.common.data.marshalling.UTF8StringSerializerDeserializer; +import org.apache.hyracks.dataflow.common.data.normalizers.IntegerNormalizedKeyComputerFactory; +import org.apache.hyracks.dataflow.common.data.normalizers.UTF8StringNormalizedKeyComputerFactory; +import org.apache.hyracks.dataflow.common.data.parsers.IValueParserFactory; +import org.apache.hyracks.dataflow.common.data.parsers.IntegerParserFactory; +import org.apache.hyracks.dataflow.common.data.parsers.UTF8StringParserFactory; +import org.apache.hyracks.dataflow.common.data.partition.FieldHashPartitionComputerFactory; +import org.apache.hyracks.dataflow.common.data.partition.range.FieldRangePartitionComputerFactory; +import org.apache.hyracks.dataflow.common.data.partition.range.IRangeMap; +import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap; +import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor; +import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor; +import org.apache.hyracks.dataflow.std.connectors.MToNReplicatingConnectorDescriptor; +import org.apache.hyracks.dataflow.std.connectors.OneToOneConnectorDescriptor; +import org.apache.hyracks.dataflow.std.file.ConstantFileSplitProvider; +import org.apache.hyracks.dataflow.std.file.DelimitedDataTupleParserFactory; +import org.apache.hyracks.dataflow.std.file.FileScanOperatorDescriptor; +import org.apache.hyracks.dataflow.std.file.FileSplit; +import org.apache.hyracks.dataflow.std.file.IFileSplitProvider; +import org.apache.hyracks.dataflow.std.file.LineFileWriteOperatorDescriptor; +import org.apache.hyracks.dataflow.std.group.HashSpillableTableFactory; +import org.apache.hyracks.dataflow.std.group.IFieldAggregateDescriptorFactory; +import org.apache.hyracks.dataflow.std.group.aggregators.AvgFieldGroupAggregatorFactory; +import org.apache.hyracks.dataflow.std.group.aggregators.AvgFieldMergeAggregatorFactory; +import org.apache.hyracks.dataflow.std.group.aggregators.CountFieldAggregatorFactory; +import org.apache.hyracks.dataflow.std.group.aggregators.IntSumFieldAggregatorFactory; +import org.apache.hyracks.dataflow.std.group.aggregators.MultiFieldsAggregatorFactory; +import org.apache.hyracks.dataflow.std.group.external.ExternalGroupOperatorDescriptor; +import org.apache.hyracks.dataflow.std.group.preclustered.PreclusteredGroupOperatorDescriptor; +import org.apache.hyracks.dataflow.std.misc.MaterializingOperatorDescriptor; +import org.apache.hyracks.dataflow.std.parallel.HistogramAlgorithm; +import org.apache.hyracks.dataflow.std.parallel.base.FieldRangePartitionDelayComputerFactory; +import org.apache.hyracks.dataflow.std.parallel.base.HistogramConnectorPolicyAssignmentPolicy; +import org.apache.hyracks.dataflow.std.parallel.histogram.AbstractSampleOperatorDescriptor; +import org.apache.hyracks.dataflow.std.parallel.histogram.MaterializingForwardOperatorDescriptor; +import org.apache.hyracks.dataflow.std.parallel.histogram.MaterializingSampleOperatorDescriptor; +import org.apache.hyracks.dataflow.std.parallel.histogram.MergeSampleOperatorDescriptor; +import org.apache.hyracks.dataflow.std.result.ResultWriterOperatorDescriptor; +import org.apache.hyracks.dataflow.std.sort.ExternalSortOperatorDescriptor; +import org.apache.hyracks.tests.util.ResultSerializerFactoryProvider; + +/** + * @author michael + */ +public class SampleForwardTest extends AbstractIntegrationTest { + private static final boolean DEBUG = false; + private static final int balance_factor = 10; + private static final int outputArity = 1; + private static final int rangeMergeArity = 1; + private static final int outputFiles = 2; + private static final int outputRaws = 2; + private static int[] sampleFields = new int[] { 2 }; + private static int[] normalFields = new int[] { 0 }; + /*private IBinaryComparatorFactory[] sampleCmpFactories = new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory + .of(IntegerPointable.FACTORY) }; + private INormalizedKeyComputerFactory sampleKeyFactories = new IntegerNormalizedKeyComputerFactory();*/ + private IBinaryComparatorFactory[] sampleCmpFactories = new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory + .of(UTF8StringPointable.FACTORY) }; + private INormalizedKeyComputerFactory sampleKeyFactories = new UTF8StringNormalizedKeyComputerFactory(); + MultiFieldsAggregatorFactory sampleAggFactory = new MultiFieldsAggregatorFactory( + new IFieldAggregateDescriptorFactory[] { new IntSumFieldAggregatorFactory(1, true) }); + + // @Test + public void sampleForward_Sample() throws Exception { + JobSpecification spec = new JobSpecification(); + File[] outputFile = new File[outputFiles]; + for (int i = 0; i < outputFiles; i++) { + outputFile[i] = File.createTempFile("output-" + i + "-", null, new File("data")); + } + File[] outputRaw = new File[outputRaws]; + for (int i = 0; i < outputRaws; i++) { + outputRaw[i] = File.createTempFile("raw-" + i + "-", null, new File("data")); + } + FileSplit[] custSplits = new FileSplit[] { + new FileSplit(NC1_ID, new FileReference(new File("data/tpch0.001/customer-part1.tbl"))), + new FileSplit(NC2_ID, new FileReference(new File("data/tpch0.001/customer-part2.tbl"))) }; + + IFileSplitProvider custSplitsProvider = new ConstantFileSplitProvider(custSplits); + RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] { + IntegerSerializerDeserializer.INSTANCE, new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), IntegerSerializerDeserializer.INSTANCE, + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer() }); + + FileScanOperatorDescriptor custScanner = new FileScanOperatorDescriptor(spec, custSplitsProvider, + new DelimitedDataTupleParserFactory(new IValueParserFactory[] { IntegerParserFactory.INSTANCE, + UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, + IntegerParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, + UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, + UTF8StringParserFactory.INSTANCE }, '|'), custDesc); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID); + + ExternalSortOperatorDescriptor sorterCust = new ExternalSortOperatorDescriptor(spec, 4, sampleFields, + sampleCmpFactories, custDesc); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorterCust, NC1_ID, NC2_ID); + spec.connect(new OneToOneConnectorDescriptor(spec), custScanner, 0, sorterCust, 0); + + AbstractSampleOperatorDescriptor materSampleCust = new MaterializingSampleOperatorDescriptor(spec, 4, + sampleFields, 2 * balance_factor, custDesc, sampleCmpFactories, null, 1, new boolean[] { true }); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, materSampleCust, NC1_ID, NC2_ID); + spec.connect(new OneToOneConnectorDescriptor(spec), sorterCust, 0, materSampleCust, 0); + + byte[] byteRange = new byte[rangeMergeArity]; + int[] offRange = new int[rangeMergeArity]; + for (int i = 0; i < rangeMergeArity; i++) { + byteRange[i] = Byte.parseByte(String.valueOf(i * (150 / rangeMergeArity + 1))); + offRange[i] = i; + } + + IRangeMap rangeMap = new RangeMap(rangeMergeArity, byteRange, offRange); + + ITuplePartitionComputerFactory tpcf = new FieldRangePartitionComputerFactory(normalFields, sampleCmpFactories, + rangeMap); + + ResultSetId rsId = new ResultSetId(1); + spec.addResultSetId(rsId); + + FileSplit[] files = new FileSplit[outputFiles]; + for (int i = 0; i < outputFiles; i++) { + files[i] = new FileSplit((0 == i % 2) ? NC1_ID : NC2_ID, new FileReference(outputFile[i])); + } + + RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] { + new UTF8StringSerializerDeserializer(), IntegerSerializerDeserializer.INSTANCE }); + IOperatorDescriptor forward = new MaterializingOperatorDescriptor(spec, outputRec); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, forward, NC1_ID, NC2_ID); + spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, tpcf, normalFields, sampleCmpFactories, + sampleKeyFactories, false), materSampleCust, 0, forward, 0); + + IOperatorDescriptor printer = new LineFileWriteOperatorDescriptor(spec, files); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID, NC2_ID); + spec.connect(new MToNReplicatingConnectorDescriptor(spec), forward, 0, printer, 0); + + ResultSetId rsRaw = new ResultSetId(2); + spec.addResultSetId(rsRaw); + FileSplit[] filesRaw = new FileSplit[outputRaws]; + for (int i = 0; i < outputRaws; i++) { + filesRaw[i] = new FileSplit((0 == i % 2) ? NC1_ID : NC2_ID, new FileReference(outputRaw[i])); + } + IOperatorDescriptor printer1 = new LineFileWriteOperatorDescriptor(spec, filesRaw); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer1, NC1_ID, NC2_ID); + spec.connect(new OneToOneConnectorDescriptor(spec), materSampleCust, 1, printer1, 0); + + spec.addRoot(printer); + spec.addRoot(printer1); + runTest(spec); + } + + // @Test + public void sampleForward_Merge() throws Exception { + JobSpecification spec = new JobSpecification(); + File[] outputFile = new File[outputFiles]; + for (int i = 0; i < outputFiles; i++) { + outputFile[i] = File.createTempFile("output-" + i + "-", null, new File("data")); + } + File[] outputRaw = new File[outputRaws]; + for (int i = 0; i < outputRaws; i++) { + outputRaw[i] = File.createTempFile("raw-" + i + "-", null, new File("data")); + } + FileSplit[] custSplits = new FileSplit[] { + new FileSplit(NC1_ID, new FileReference(new File("data/tpch0.001/customer-part1.tbl"))), + new FileSplit(NC2_ID, new FileReference(new File("data/tpch0.001/customer-part2.tbl"))) }; + IFileSplitProvider custSplitsProvider = new ConstantFileSplitProvider(custSplits); + RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] { + IntegerSerializerDeserializer.INSTANCE, new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), IntegerSerializerDeserializer.INSTANCE, + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer() }); + + FileScanOperatorDescriptor custScanner = new FileScanOperatorDescriptor(spec, custSplitsProvider, + new DelimitedDataTupleParserFactory(new IValueParserFactory[] { IntegerParserFactory.INSTANCE, + UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, + IntegerParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, + UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, + UTF8StringParserFactory.INSTANCE }, '|'), custDesc); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID); + + ExternalSortOperatorDescriptor sorterCust = new ExternalSortOperatorDescriptor(spec, 4, sampleFields, + sampleCmpFactories, custDesc); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorterCust, NC1_ID, NC2_ID); + spec.connect(new OneToOneConnectorDescriptor(spec), custScanner, 0, sorterCust, 0); + + AbstractSampleOperatorDescriptor materSampleCust = new MaterializingSampleOperatorDescriptor(spec, 4, + sampleFields, 2 * balance_factor, custDesc, sampleCmpFactories, null, 1, new boolean[] { true }); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, materSampleCust, NC1_ID, NC2_ID); + spec.connect(new OneToOneConnectorDescriptor(spec), sorterCust, 0, materSampleCust, 0); + + RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] { + new UTF8StringSerializerDeserializer(), IntegerSerializerDeserializer.INSTANCE }); + + byte[] byteRange = new byte[rangeMergeArity]; + int[] offRange = new int[rangeMergeArity]; + for (int i = 0; i < rangeMergeArity; i++) { + byteRange[i] = Byte.parseByte(String.valueOf(i * (150 / rangeMergeArity + 1))); + offRange[i] = i; + } + + IRangeMap rangeMap = new RangeMap(rangeMergeArity, byteRange, offRange); + + ITuplePartitionComputerFactory tpcf = new FieldRangePartitionComputerFactory(normalFields, sampleCmpFactories, + rangeMap); + + IOperatorDescriptor mergeSampleCust = new MergeSampleOperatorDescriptor(spec, 4, normalFields, outputRec, 4, + sampleKeyFactories, sampleCmpFactories, HistogramAlgorithm.ORDERED_HISTOGRAM, false); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, mergeSampleCust, NC1_ID, NC2_ID); + spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, tpcf, normalFields, sampleCmpFactories, + sampleKeyFactories, false), materSampleCust, 0, mergeSampleCust, 0); + + ResultSetId rsId = new ResultSetId(1); + spec.addResultSetId(rsId); + + FileSplit[] files = new FileSplit[outputFiles]; + for (int i = 0; i < outputFiles; i++) { + files[i] = new FileSplit((0 == i % 2) ? NC1_ID : NC2_ID, new FileReference(outputFile[i])); + } + + IOperatorDescriptor printer = new LineFileWriteOperatorDescriptor(spec, files); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID, NC2_ID); + spec.connect(new MToNReplicatingConnectorDescriptor(spec), mergeSampleCust, 0, printer, 0); + + ResultSetId rsRaw = new ResultSetId(2); + spec.addResultSetId(rsRaw); + FileSplit[] filesRaw = new FileSplit[outputRaws]; + for (int i = 0; i < outputRaws; i++) { + filesRaw[i] = new FileSplit((0 == i % 2) ? NC1_ID : NC2_ID, new FileReference(outputRaw[i])); + } + IOperatorDescriptor printer1 = new LineFileWriteOperatorDescriptor(spec, filesRaw); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer1, NC1_ID, NC2_ID); + spec.connect(new OneToOneConnectorDescriptor(spec), materSampleCust, 1, printer1, 0); + + spec.addRoot(printer); + spec.addRoot(printer1); + runTest(spec); + } + + // @Test + public void sampleForward_Total() throws Exception { + JobSpecification spec = new JobSpecification(); + File[] outputFile = new File[outputFiles]; + for (int i = 0; i < outputFiles; i++) { + outputFile[i] = File.createTempFile("output-" + i + "-", null, new File("data")); + } + FileSplit[] custSplits = new FileSplit[] { + new FileSplit(NC1_ID, new FileReference(new File("data/tpch0.001/customer-part1.tbl"))), + new FileSplit(NC2_ID, new FileReference(new File("data/tpch0.001/customer-part1.tbl"))) }; + IFileSplitProvider custSplitsProvider = new ConstantFileSplitProvider(custSplits); + RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] { + IntegerSerializerDeserializer.INSTANCE, new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), IntegerSerializerDeserializer.INSTANCE, + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer() }); + + FileScanOperatorDescriptor custScanner = new FileScanOperatorDescriptor(spec, custSplitsProvider, + new DelimitedDataTupleParserFactory(new IValueParserFactory[] { IntegerParserFactory.INSTANCE, + UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, + IntegerParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, + UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, + UTF8StringParserFactory.INSTANCE }, '|'), custDesc); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID); + + ExternalSortOperatorDescriptor sorterCust = new ExternalSortOperatorDescriptor(spec, 4, sampleFields, + sampleCmpFactories, custDesc); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorterCust, NC1_ID, NC2_ID); + spec.connect(new OneToOneConnectorDescriptor(spec), custScanner, 0, sorterCust, 0); + + AbstractSampleOperatorDescriptor materSampleCust = new MaterializingSampleOperatorDescriptor(spec, 4, + sampleFields, 2 * balance_factor, custDesc, sampleCmpFactories, null, 1, new boolean[] { true }); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, materSampleCust, NC1_ID, NC2_ID); + spec.connect(new OneToOneConnectorDescriptor(spec), sorterCust, 0, materSampleCust, 0); + + RecordDescriptor outputSamp = new RecordDescriptor(new ISerializerDeserializer[] { + IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE }); + + byte[] byteRange = new byte[rangeMergeArity]; + int[] offRange = new int[rangeMergeArity]; + for (int i = 0; i < rangeMergeArity; i++) { + byteRange[i] = Byte.parseByte(String.valueOf(i * (150 / rangeMergeArity + 1))); + offRange[i] = i; + } + + IRangeMap rangeMap = new RangeMap(normalFields.length, byteRange, offRange); + + ITuplePartitionComputerFactory tpcf = new FieldRangePartitionComputerFactory(normalFields, sampleCmpFactories, + rangeMap); + + IOperatorDescriptor mergeSampleCust = new MergeSampleOperatorDescriptor(spec, 4, normalFields, outputSamp, 4, + sampleKeyFactories, sampleCmpFactories, HistogramAlgorithm.ORDERED_HISTOGRAM, false); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, mergeSampleCust, NC1_ID, NC2_ID); + spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, tpcf, normalFields, sampleCmpFactories, + sampleKeyFactories, false), materSampleCust, 0, mergeSampleCust, 0); + spec.setConnectorPolicyAssignmentPolicy(new ConnectorPolicyAssignmentPolicy()); + + ITuplePartitionComputerFactory tpc = new FieldRangePartitionDelayComputerFactory(sampleFields, + sampleCmpFactories); + + RecordDescriptor outputRec = custDesc; + IOperatorDescriptor forward = new MaterializingForwardOperatorDescriptor(spec, 4, normalFields, outputSamp, + outputRec, sampleCmpFactories); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, forward, NC1_ID, NC2_ID); + spec.connect(new MToNReplicatingConnectorDescriptor(spec), mergeSampleCust, 0, forward, 0); + spec.connect(new OneToOneConnectorDescriptor(spec), materSampleCust, 1, forward, 1); + + ResultSetId rsId = new ResultSetId(1); + spec.addResultSetId(rsId); + + FileSplit[] files = new FileSplit[outputFiles]; + for (int i = 0; i < outputFiles; i++) { + files[i] = new FileSplit((0 == i % 2) ? NC1_ID : NC2_ID, new FileReference(outputFile[i])); + } + + /*byteRange = new byte[32]; + offRange = new int[4]; + int current = 0; + for (int i = 0; i < 4; i++) { + offRange[i] = current; + byte[] tick = ByteBuffer.allocate(4).putInt(i * (15000 / 4 + 1)).array(); + for (int j = 0; j < tick.length; j++) { + byteRange[current + j] = tick[j]; + } + current += tick.length;; + } + rangeMap = new RangeMap(normalFields.length, byteRange, offRange); + tpcf = new FieldRangePartitionComputerFactory(normalFields, sampleCmpFactories, rangeMap);*/ + + IOperatorDescriptor printer = new LineFileWriteOperatorDescriptor(spec, files); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID, NC2_ID); + spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, tpc, sampleFields, sampleCmpFactories, + sampleKeyFactories, false), forward, 0, printer, 0); + + spec.addRoot(printer); + runTest(spec); + } + + @Test + public void sampleSort() throws Exception { + JobSpecification spec = new JobSpecification(); + File[] outputFile = new File[outputFiles]; + for (int i = 0; i < outputFiles; i++) { + outputFile[i] = File.createTempFile("output-" + i + "-", null, new File("data")); + } + FileSplit[] custSplits = new FileSplit[] { + new FileSplit(NC1_ID, new FileReference(new File("data/tpch0.001/customer-part1.tbl"))), + new FileSplit(NC2_ID, new FileReference(new File("data/tpch0.001/customer-part2.tbl"))) }; + IFileSplitProvider custSplitsProvider = new ConstantFileSplitProvider(custSplits); + RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] { + IntegerSerializerDeserializer.INSTANCE, new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), IntegerSerializerDeserializer.INSTANCE, + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer() }); + + FileScanOperatorDescriptor custScanner = new FileScanOperatorDescriptor(spec, custSplitsProvider, + new DelimitedDataTupleParserFactory(new IValueParserFactory[] { IntegerParserFactory.INSTANCE, + UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, + IntegerParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, + UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, + UTF8StringParserFactory.INSTANCE }, '|'), custDesc); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID); + + AbstractSampleOperatorDescriptor materSampleCust = new MaterializingSampleOperatorDescriptor(spec, 4, + sampleFields, 2 * balance_factor, custDesc, sampleCmpFactories, null, 1, new boolean[] { true }); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, materSampleCust, NC1_ID, NC2_ID); + spec.connect(new OneToOneConnectorDescriptor(spec), custScanner, 0, materSampleCust, 0); + + RecordDescriptor outputSamp = new RecordDescriptor(new ISerializerDeserializer[] { + IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE }); + + byte[] byteRange = new byte[rangeMergeArity]; + int[] offRange = new int[rangeMergeArity]; + for (int i = 0; i < rangeMergeArity; i++) { + byteRange[i] = Byte.parseByte(String.valueOf(i * (150 / rangeMergeArity + 1))); + offRange[i] = i; + } + + IRangeMap rangeMap = new RangeMap(normalFields.length, byteRange, offRange); + + ITuplePartitionComputerFactory tpcf = new FieldRangePartitionComputerFactory(normalFields, sampleCmpFactories, + rangeMap); + + IOperatorDescriptor mergeSampleCust = new MergeSampleOperatorDescriptor(spec, 4, normalFields, outputSamp, 4, + sampleKeyFactories, sampleCmpFactories, HistogramAlgorithm.ORDERED_HISTOGRAM, false); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, mergeSampleCust, NC1_ID, NC2_ID); + spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, tpcf, normalFields, sampleCmpFactories, + sampleKeyFactories, false), materSampleCust, 0, mergeSampleCust, 0); + + ITuplePartitionComputerFactory tpc = new FieldRangePartitionDelayComputerFactory(sampleFields, + sampleCmpFactories); + + RecordDescriptor outputRec = custDesc; + IOperatorDescriptor forward = new MaterializingForwardOperatorDescriptor(spec, 4, normalFields, outputSamp, + outputRec, sampleCmpFactories); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, forward, NC1_ID, NC2_ID); + spec.connect(new MToNReplicatingConnectorDescriptor(spec), mergeSampleCust, 0, forward, 0); + spec.connect(new OneToOneConnectorDescriptor(spec), materSampleCust, 1, forward, 1); + + ExternalSortOperatorDescriptor sorterCust = new ExternalSortOperatorDescriptor(spec, 4, sampleFields, + sampleCmpFactories, custDesc); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorterCust, NC1_ID, NC2_ID); + spec.connect(new MToNPartitioningConnectorDescriptor(spec, tpc), forward, 0, sorterCust, 0); + + ResultSetId rsId = new ResultSetId(1); + spec.addResultSetId(rsId); + + FileSplit[] files = new FileSplit[outputFiles]; + for (int i = 0; i < outputFiles; i++) { + files[i] = new FileSplit((0 == i % 2) ? NC1_ID : NC2_ID, new FileReference(outputFile[i])); + } + + IOperatorDescriptor printer = new LineFileWriteOperatorDescriptor(spec, files); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID, NC2_ID); + spec.connect(new OneToOneConnectorDescriptor(spec), sorterCust, 0, printer, 0); + + spec.addRoot(printer); + runTest(spec); + } + + // @Test + public void sampleForward_Case1() throws Exception { + JobSpecification spec = new JobSpecification(); + File[] outputFile = new File[outputFiles]; + for (int i = 0; i < outputFiles; i++) { + outputFile[i] = File.createTempFile("output-" + i + "-", null, new File("data")); + } + File[] outputRaw = new File[outputRaws]; + for (int i = 0; i < outputRaws; i++) { + outputRaw[i] = File.createTempFile("raw-" + i + "-", null, new File("data")); + } + FileSplit[] custSplits = new FileSplit[] { + new FileSplit(NC1_ID, new FileReference(new File("data/tpch0.001/customer-part1.tbl"))), + new FileSplit(NC2_ID, new FileReference(new File("data/tpch0.001/customer-part2.tbl"))) }; + // FileSplit[] custSplits = new FileSplit[] { + // new FileSplit(NC1_ID, new FileReference(new File("data/tpch0.001/customer-part1.tbl"))), + // new FileSplit(NC2_ID, new FileReference(new File("data/tpch0.001/customer-part2.tbl"))) }; + IFileSplitProvider custSplitsProvider = new ConstantFileSplitProvider(custSplits); + RecordDescriptor custDesc = new RecordDescriptor(new ISerializerDeserializer[] { + IntegerSerializerDeserializer.INSTANCE, new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), IntegerSerializerDeserializer.INSTANCE, + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer() }); + + FileSplit[] ordersSplits = new FileSplit[] { + new FileSplit(NC1_ID, new FileReference(new File("data/tpch0.001/orders-part1.tbl"))), + new FileSplit(NC2_ID, new FileReference(new File("data/tpch0.001/orders-part2.tbl"))) }; + + IFileSplitProvider ordersSplitsProvider = new ConstantFileSplitProvider(ordersSplits); + RecordDescriptor ordersDesc = new RecordDescriptor(new ISerializerDeserializer[] { + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer() }); + + RecordDescriptor custOrderJoinDesc = new RecordDescriptor(new ISerializerDeserializer[] { + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer(), new UTF8StringSerializerDeserializer(), + new UTF8StringSerializerDeserializer() }); + + /*FileScanOperatorDescriptor ordScanner = new FileScanOperatorDescriptor(spec, ordersSplitsProvider, + new DelimitedDataTupleParserFactory(new IValueParserFactory[] { UTF8StringParserFactory.INSTANCE, + UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, + UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, + UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, + UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE }, '|'), ordersDesc); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, ordScanner, NC1_ID, NC2_ID);*/ + + FileScanOperatorDescriptor custScanner = new FileScanOperatorDescriptor(spec, custSplitsProvider, + new DelimitedDataTupleParserFactory(new IValueParserFactory[] { IntegerParserFactory.INSTANCE, + UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, + IntegerParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, + UTF8StringParserFactory.INSTANCE, UTF8StringParserFactory.INSTANCE, + UTF8StringParserFactory.INSTANCE }, '|'), custDesc); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, custScanner, NC1_ID, NC2_ID); + + /*ExternalSortOperatorDescriptor sorterOrd = new ExternalSortOperatorDescriptor(spec, 4, new int[] { 1 }, + new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), + PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }, ordersDesc); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorterOrd, NC1_ID, NC2_ID); + spec.connect(new OneToOneConnectorDescriptor(spec), ordScanner, 0, sorterOrd, 0);*/ + + ExternalSortOperatorDescriptor sorterCust = new ExternalSortOperatorDescriptor(spec, 4, sampleFields, + sampleCmpFactories, custDesc); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, sorterCust, NC1_ID, NC2_ID); + spec.connect(new OneToOneConnectorDescriptor(spec), custScanner, 0, sorterCust, 0); + + /*MaterializingOperatorDescriptor materOrd = new MaterializingOperatorDescriptor(spec, ordersDesc); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, materOrd, NC1_ID, NC2_ID); + spec.connect(new OneToOneConnectorDescriptor(spec), sorterOrd, 0, materOrd, 0);*/ + + AbstractSampleOperatorDescriptor materSampleCust = new MaterializingSampleOperatorDescriptor(spec, 4, + sampleFields, 2 * balance_factor, custDesc, sampleCmpFactories, null, 1, new boolean[] { true }); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, materSampleCust, NC1_ID, NC2_ID); + spec.connect(new OneToOneConnectorDescriptor(spec), sorterCust, 0, materSampleCust, 0); + + /*ITuplePartitionComputerFactory tpcf = new FieldRangePartitionComputerFactory(new int[] { 0 }, + new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), + PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }, rangeMap); + spec.connect(new MToNPartitioningConnectorDescriptor(spec, tpcf), sorterCust, 0, materSampleCust, 0);*/ + + // spec.connect( + // new MToNPartitioningConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(new int[] { 1, 0 }, + // new IBinaryHashFunctionFactory[] { + // PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY), + // PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) })), sorterCust, 0, + // materSampleCust, 0); + + /*spec.connect( + new MToNPartitioningMergingConnectorDescriptor(spec, new FieldHashPartitionComputerFactory(new int[] { + 1, 0 }, new IBinaryHashFunctionFactory[] { + PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY), + PointableBinaryHashFunctionFactory.of(UTF8StringPointable.FACTORY) }), new int[] { 1, 0 }, + new IBinaryComparatorFactory[] { + PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY), + PointableBinaryComparatorFactory.of(UTF8StringPointable.FACTORY) }, + new UTF8StringNormalizedKeyComputerFactory()), sorterCust, 0, materSampleCust, 0);*/ + + // MaterializingOperatorDescriptor materCust = new MaterializingOperatorDescriptor(spec, custDesc); + // PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, materCust, NC1_ID, NC2_ID); + // spec.connect(new OneToOneConnectorDescriptor(spec), sorterCust, 0, materCust, 0); + + // + // ITuplePartitionComputerFactory tpcf = new FieldRangePartitionComputerFactory(offRange, + // new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, + // rangeMap); + // spec.connect(new MToNPartitioningConnectorDescriptor(spec, tpcf), sorterCust, 0, materCust, 0); + + RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] { + new UTF8StringSerializerDeserializer(), IntegerSerializerDeserializer.INSTANCE }); + // RecordDescriptor outputRec = new RecordDescriptor(new ISerializerDeserializer[] { + // IntegerSerializerDeserializer.INSTANCE, IntegerSerializerDeserializer.INSTANCE }); + + /*ExternalGroupOperatorDescriptor grouper = new ExternalGroupOperatorDescriptor(spec, sampleFields, 4, + sampleCmpFactories, sampleKeyFactories, sampleAggFactory, sampleAggFactory, outputRec, null, false);*/ + + // PreclusteredGroupOperatorDescriptor grouper = new PreclusteredGroupOperatorDescriptor(spec, sampleFields, + // sampleCmpFactories, sampleAggFactory, outputRec); + // PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, grouper, NC1_ID, NC2_ID); + // spec.connect(new OneToOneConnectorDescriptor(spec), sorterCust, 0, grouper, 0); + + byte[] byteRange = new byte[rangeMergeArity]; + int[] offRange = new int[rangeMergeArity]; + for (int i = 0; i < rangeMergeArity; i++) { + byteRange[i] = Byte.parseByte(String.valueOf(i * (150 / rangeMergeArity + 1))); + offRange[i] = i; + } + + IRangeMap rangeMap = new RangeMap(rangeMergeArity, byteRange, offRange); + + ITuplePartitionComputerFactory tpcf = new FieldRangePartitionComputerFactory(normalFields, sampleCmpFactories, + rangeMap); + // spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, tpcf, normalFields, sampleCmpFactories, + // sampleKeyFactories, false), materSampleCust, 0, grouper, 0); + + // spec.setConnectorPolicyAssignmentPolicy(new SampleConnectorPolicyAssignmentPolicy()); + // spec.setUseConnectorPolicyForScheduling(true); + // spec.setConnectorPolicyAssignmentPolicy(IConnectorPolicyAssignmentPolicy); // IConnectorPolicy + // spec.connect(new MToNPartitioningConnectorDescriptor(spec, tpcf), materSampleCust, 0, grouper, 0); + + // spec.connect(new MToNReplicatingConnectorDescriptor(spec), grouper, 0, materSampleCust, 1); + + IOperatorDescriptor mergeSampleCust = new MergeSampleOperatorDescriptor(spec, 4, normalFields, outputRec, 4, + sampleKeyFactories, sampleCmpFactories, HistogramAlgorithm.ORDERED_HISTOGRAM, false); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, mergeSampleCust, NC1_ID, NC2_ID); + spec.connect(new MToNPartitioningMergingConnectorDescriptor(spec, tpcf, normalFields, sampleCmpFactories, + sampleKeyFactories, false), materSampleCust, 0, mergeSampleCust, 0); + + ResultSetId rsId = new ResultSetId(1); + spec.addResultSetId(rsId); + + FileSplit[] files = new FileSplit[outputFiles]; + for (int i = 0; i < outputFiles; i++) { + files[i] = new FileSplit((0 == i % 2) ? NC1_ID : NC2_ID, new FileReference(outputFile[i])); + } + + IOperatorDescriptor printer = new LineFileWriteOperatorDescriptor(spec, files); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID, NC2_ID); + spec.connect(new MToNReplicatingConnectorDescriptor(spec), mergeSampleCust, 0, printer, 0); + // spec.connect(new OneToOneConnectorDescriptor(spec), materSampleCust, 0, printer, 0); + + ResultSetId rsRaw = new ResultSetId(2); + spec.addResultSetId(rsRaw); + FileSplit[] filesRaw = new FileSplit[outputRaws]; + for (int i = 0; i < outputRaws; i++) { + filesRaw[i] = new FileSplit((0 == i % 2) ? NC1_ID : NC2_ID, new FileReference(outputRaw[i])); + } + + IOperatorDescriptor printer1 = new LineFileWriteOperatorDescriptor(spec, filesRaw); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer1, NC1_ID, NC2_ID); + spec.connect(new OneToOneConnectorDescriptor(spec), materSampleCust, 1, printer1, 0); + + // spec.connect(new OneToOneConnectorDescriptor(spec), materSampleCust, 1, printer, 0); + /*byte[] byteRange = new byte[outputFiles]; + int[] offRange = new int[outputFiles]; + for (int i = 0; i < outputFiles; i++) { + byteRange[i] = Byte.parseByte(String.valueOf(i * 40)); + offRange[i] = i; + } + IRangeMap rangeMap = new RangeMap(1, byteRange, offRange); + ITuplePartitionComputerFactory tpcf = new FieldRangePartitionComputerFactory(new int[] { 0 }, + new IBinaryComparatorFactory[] { PointableBinaryComparatorFactory.of(IntegerPointable.FACTORY) }, + rangeMap); + spec.connect(new MToNPartitioningConnectorDescriptor(spec, tpcf), materCust, 0, printer, 0); + + ResultSetId rsId = new ResultSetId(1); + spec.addResultSetId(rsId);*/ + + /*IOperatorDescriptor printer = new ResultWriterOperatorDescriptor(spec, rsId, false, false, + ResultSerializerFactoryProvider.INSTANCE.getResultSerializerFactoryProvider()); + PartitionConstraintHelper.addAbsoluteLocationConstraint(spec, printer, NC1_ID, NC2_ID); + spec.connect(new OneToOneConnectorDescriptor(spec), materSampleCust, 0, printer, 0);*/ + + spec.addRoot(printer); + spec.addRoot(printer1); + runTest(spec); + } +} diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/SamplerTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/SamplerTest.java new file mode 100644 index 00000000000..8b3d62006df --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/integration/SamplerTest.java @@ -0,0 +1,118 @@ +/* + * 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.hyracks.tests.integration; + +import java.io.BufferedReader; +import java.io.FileReader; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.logging.Logger; + +import junit.framework.Assert; + +import org.apache.hyracks.dataflow.std.parallel.sampler.ISampler; +import org.apache.hyracks.dataflow.std.parallel.sampler.ReservoirSampler; +import org.junit.Test; + +public class SamplerTest { + private static final Logger LOGGER = Logger.getLogger(SamplerTest.class.getName()); + + private static final String filePath = "data/skew/zipfan.tbl"; + + private final static int ZIPFAN_COLUMN = 0; + + private final static int SAMPLE_COUNT = 4000; + + private final static double ERROR_BOUND = 0.2; + + private final static int PARTITION_COUNT = 8; + + private int totalCount = 0; + + private final static Map randDatum = new HashMap(); + + @Test + public void testSampler() throws Exception { + BufferedReader br = new BufferedReader(new FileReader(filePath)); + String line = null; + while (null != (line = br.readLine())) { + String[] fields = line.split("\t"); + String strD = fields[ZIPFAN_COLUMN]; + randDatum.put((int) (Math.random() * 1000000000), Double.parseDouble(strD)); + totalCount++; + } + br.close(); + + ISampler sampler = new ReservoirSampler(SAMPLE_COUNT, true); + + Iterator> iter = randDatum.entrySet().iterator(); + while (iter.hasNext()) + sampler.sample(iter.next().getValue()); + + Iterator sampleed = sampler.getSamples().iterator(); + LOGGER.info("Total sampled: " + sampler.getSize()); + List rangeMap = new ArrayList(); + int[] count = new int[PARTITION_COUNT]; + for (int i = 0; i < PARTITION_COUNT; i++) + count[i] = 0; + int current = 0; + while (sampleed.hasNext()) { + ++current; + if (current % (SAMPLE_COUNT / PARTITION_COUNT) == 0) + rangeMap.add(sampleed.next()); + else + sampleed.next(); + } + LOGGER.info("rangeMap: " + rangeMap.size() + " actual: " + current); + if (rangeMap.size() == PARTITION_COUNT) + rangeMap.remove(rangeMap.size() - 1); + + iter = randDatum.entrySet().iterator(); + + while (iter.hasNext()) { + double value = iter.next().getValue(); + boolean found = false; + for (int i = 0; i < rangeMap.size(); i++) { + if (rangeMap.get(i) > value) { + count[i]++; + found = true; + break; + } + } + if (!found) + count[count.length - 1]++; + } + + int cMax = 0; + for (int i = 0; i < count.length - 1; i++) { + LOGGER.info(rangeMap.get(i) + " <-> " + count[i]); + } + LOGGER.info("INF <-> " + count[count.length - 1]); + for (int i = 0; i < count.length; i++) { + if (cMax < count[i]) + cMax = count[i]; + } + Assert.assertEquals(0, + (int) ((cMax - (totalCount / PARTITION_COUNT)) / ((double) totalCount / PARTITION_COUNT * ERROR_BOUND))); + } +} diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/util/SampleAlgorithmTests.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/util/SampleAlgorithmTests.java new file mode 100644 index 00000000000..5c2fdb767c9 --- /dev/null +++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/util/SampleAlgorithmTests.java @@ -0,0 +1,898 @@ +/* + * 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.hyracks.tests.util; + +import java.io.BufferedReader; +import java.io.BufferedWriter; +import java.io.ByteArrayOutputStream; +import java.io.FileReader; +import java.io.FileWriter; +import java.io.IOException; +import java.io.PrintStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.BitSet; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.logging.Logger; + +import junit.framework.TestCase; + +import org.apache.hyracks.data.std.primitive.DoublePointable; +import org.apache.hyracks.data.std.primitive.UTF8StringPointable; +import org.apache.hyracks.dataflow.std.parallel.IHistogram; +import org.apache.hyracks.dataflow.std.parallel.IIterativeHistogram; +import org.apache.hyracks.dataflow.std.parallel.histogram.structures.DTStreamingHistogram; +import org.apache.hyracks.dataflow.std.parallel.histogram.structures.TernaryIterativeHistogram; +import org.apache.hyracks.dataflow.std.parallel.util.DualSerialEntry; +import org.apache.hyracks.dataflow.std.parallel.util.HistogramUtils; +import org.apache.hyracks.util.string.UTF8StringUtil; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +/** + * @author michael + */ +public class SampleAlgorithmTests extends TestCase { + private static final Logger LOGGER = Logger.getLogger(SampleAlgorithmTests.class.getName()); + private final ByteArrayOutputStream outContent = new ByteArrayOutputStream(); + private final ByteArrayOutputStream errContent = new ByteArrayOutputStream(); + private final static int DEFAULT_COLUMN = 5; + //private final static int ADDRESS_COLUMN = 2; + private final static int COMMENT_COLUMN = 15; + private final static int ZIPFAN_COLUMN = 0; + private/*final static*/int PARTITION_CARD = 7; + private final static double precision = 0.01; + private final static boolean fixPointable = false; + private final static boolean printQuantiles = false; + private final static boolean deeper = true; + private final int sampleJump = 1; + private final static boolean randomSample = true; + private final static boolean coveredTest = false; + + private final static String filePath = "data/tpch0.001/lineitem.tbl"; + + //private final static String filePath = "/Users/michael/Desktop/tpch_2_16_1/dbgen/lineitem.tbl"; + + @Before + public void setUpStreams() { + System.setOut(new PrintStream(outContent)); + System.setErr(new PrintStream(errContent)); + } + + @SuppressWarnings("unused") + @Test + public void testIterativeTernaryHitogram() throws Exception { + IHistogram tih = new TernaryIterativeHistogram(PARTITION_CARD, precision, fixPointable, + deeper); + tih.initialize(); + BufferedReader br = new BufferedReader(new FileReader(filePath)); + String line = null; + int total = 0; + int round = 0; + + //Sampling histogram + long begin = System.currentTimeMillis(); + BitSet sampledBits = new BitSet(); + long start = System.currentTimeMillis(); + while (null != (line = br.readLine())) { + if (randomSample) { + if (sampleJump > 1 && Math.round(Math.random() * (double) sampleJump) != 4) { + total++; + continue; + } else + total++; + } else { + if (total++ % sampleJump != 0) { + continue; + } + } + sampledBits.set(total); + String[] fields = line.split("\\|"); + UTF8StringPointable key = (UTF8StringPointable) UTF8StringPointable.FACTORY.createPointable(); + String strD = fields[COMMENT_COLUMN]; + byte[] buf = HistogramUtils.toUTF8Byte(strD/*.toCharArray()*/, 0); + key.set(buf, 0, UTF8StringUtil.getUTFLength(buf, 0)); + tih.addItem(key); + } + LOGGER.info("Round" + round + " elipse: " + (System.currentTimeMillis() - start) + " "); + boolean need = ((IIterativeHistogram) tih).needIteration(); + while (need) { + start = System.currentTimeMillis(); + int current = 0; + ((IIterativeHistogram) tih).disperse(); + round++; + LOGGER.info("Round" + round + " elipse: " + (System.currentTimeMillis() - start) + " "); + need = ((IIterativeHistogram) tih).needIteration(); + } + + //Sequential Merge + List> generated = tih.generate(true); + String quantileOut = ""; + + quantileOut = ""; + for (int i = 0; i < generated.size(); i++) { + StringBuilder sb = new StringBuilder(); + UTF8StringUtil.toString(sb, generated.get(i).getKey().getByteArray(), 0); + quantileOut += ("<" + sb.toString() + ", " + generated.get(i).getValue() + ">\n"); + } + LOGGER.info(quantileOut); + quantileOut = ""; + long end = System.currentTimeMillis(); + LOGGER.info("fixed before clipse: " + (end - begin)); + if (fixPointable) { + List> fixed = ((IIterativeHistogram) tih) + .getFixPointable(); + for (int i = 0; i < fixed.size(); i++) { + StringBuilder sb = new StringBuilder(); + UTF8StringUtil.toString(sb, fixed.get(i).getKey().getByteArray(), 0); + quantileOut += ("<" + sb.toString() + ", " + fixed.get(i).getValue() + ">\n"); + } + } + LOGGER.info(quantileOut); + quantileOut = ""; + //Verification + LOGGER.info("Verification:"); + + List ticks = new ArrayList(); + List counts = new ArrayList(); + for (int i = 0; i < generated.size(); i++) { + StringBuilder lastString = new StringBuilder(); + UTF8StringUtil.toString(lastString, generated.get(i).getKey().getByteArray(), 0); + ticks.add(lastString.toString()); + counts.add(0); + } + + br.close(); + br = new BufferedReader(new FileReader(filePath)); + Map strLengthMap = new HashMap(); + if (counts.size() > 0) { + while (null != (line = br.readLine())) { + String[] fields = line.split("\\|"); + String strD = fields[COMMENT_COLUMN]; + if (coveredTest) { + Integer len = strD.length(); + if (strLengthMap.containsKey(len)) + strLengthMap.put(len, strLengthMap.get(len) + 1); + else + strLengthMap.put(len, 1); + } + boolean isLast = true; + for (int i = 0; i < ticks.size() - 1; i++) { + if (ticks.get(i).compareTo(strD) >= 0) { + counts.set(i, counts.get(i) + 1); + isLast = false; + break; + } + } + if (isLast) + counts.set(counts.size() - 1, counts.get(counts.size() - 1) + 1); + } + } + + for (int i = 0; i < ticks.size(); i++) { + quantileOut += ("[" + ticks.get(i) + ", " + counts.get(i) + "]\n"); + } + + LOGGER.info(quantileOut); + for (Entry e : strLengthMap.entrySet()) + LOGGER.info("length: " + e.getKey() + " has: " + e.getValue()); + + br.close(); + } + + @SuppressWarnings("unused") + @Test + public void testTernaryHistogram() throws Exception { + IHistogram tih = new TernaryIterativeHistogram(PARTITION_CARD, precision, fixPointable, + false); + tih.initialize(); + BufferedReader br = new BufferedReader(new FileReader(filePath)); + String line = null; + int total = 0; + int round = 1; + + //Sampling histogram + long begin = System.currentTimeMillis(); + BitSet sampledBits = new BitSet(); + long start = System.currentTimeMillis(); + while (null != (line = br.readLine())) { + if (randomSample) { + if (sampleJump > 1 && Math.round(Math.random() * (double) sampleJump) != 4) { + total++; + continue; + } else + total++; + } else { + if (total++ % sampleJump != 0) { + continue; + } + } + sampledBits.set(total); + String[] fields = line.split("\\|"); + UTF8StringPointable key = (UTF8StringPointable) UTF8StringPointable.FACTORY.createPointable(); + String strD = fields[COMMENT_COLUMN]; + byte[] buf = HistogramUtils.toUTF8Byte(strD/*.toCharArray()*/, 0); + key.set(buf, 0, UTF8StringUtil.getUTFLength(buf, 0)); + tih.addItem(key); + } + LOGGER.info("Round" + round + " elipse: " + (System.currentTimeMillis() - begin)); + begin = System.currentTimeMillis(); + boolean need = ((IIterativeHistogram) tih).needIteration(); + while (need) { + start = System.currentTimeMillis(); + int current = 0; + tih.initialize(); + br.close(); + br = new BufferedReader(new FileReader(filePath)); + while (null != (line = br.readLine())) { + if (!sampledBits.get(current++)) + continue; + String[] fields = line.split("\\|"); + UTF8StringPointable key = (UTF8StringPointable) UTF8StringPointable.FACTORY.createPointable(); + String strD = fields[COMMENT_COLUMN]; + byte[] buf = HistogramUtils.toUTF8Byte(strD/*.toCharArray()*/, 0); + key.set(buf, 0, UTF8StringUtil.getUTFLength(buf, 0)); + tih.addItem(key); + } + round++; + need = ((IIterativeHistogram) tih).needIteration(); + LOGGER.info("Round" + round + " elipse: " + (System.currentTimeMillis() - begin)); + begin = System.currentTimeMillis(); + } + + //Sequential Merge + String quantileOut = ""; + List> generated = tih.generate(true); + + quantileOut = ""; + for (int i = 0; i < generated.size(); i++) { + StringBuilder sb = new StringBuilder(); + UTF8StringUtil.toString(sb, generated.get(i).getKey().getByteArray(), 0); + quantileOut += ("<" + sb.toString() + ", " + generated.get(i).getValue() + ">\n"); + } + LOGGER.info(quantileOut); + + quantileOut = ""; + long end = System.currentTimeMillis(); + LOGGER.info("fixed before clipse: " + (end - begin)); + if (fixPointable) { + List> fixed = ((IIterativeHistogram) tih) + .getFixPointable(); + for (int i = 0; i < fixed.size(); i++) { + StringBuilder sb = new StringBuilder(); + UTF8StringUtil.toString(sb, fixed.get(i).getKey().getByteArray(), 0); + quantileOut += ("<" + sb.toString() + ", " + fixed.get(i).getValue() + ">\n"); + } + } + LOGGER.info(quantileOut); + + //Verification + LOGGER.info("Verification"); + + List ticks = new ArrayList(); + List counts = new ArrayList(); + for (int i = 0; i < generated.size(); i++) { + StringBuilder lastString = new StringBuilder(); + UTF8StringUtil.toString(lastString, generated.get(i).getKey().getByteArray(), 0); + ticks.add(lastString.toString()); + counts.add(0); + } + + br.close(); + br = new BufferedReader(new FileReader(filePath)); + Map strLengthMap = new HashMap(); + if (counts.size() > 0) { + while (null != (line = br.readLine())) { + String[] fields = line.split("\\|"); + String strD = fields[COMMENT_COLUMN]; + if (coveredTest) { + Integer len = strD.length(); + if (strLengthMap.containsKey(len)) + strLengthMap.put(len, strLengthMap.get(len) + 1); + else + strLengthMap.put(len, 1); + } + boolean isLast = true; + for (int i = 0; i < ticks.size() - 1; i++) { + if (ticks.get(i).compareTo(strD) >= 0) { + counts.set(i, counts.get(i) + 1); + isLast = false; + break; + } + } + if (isLast) + counts.set(counts.size() - 1, counts.get(counts.size() - 1) + 1); + } + } + + quantileOut = ""; + for (int i = 0; i < ticks.size(); i++) { + quantileOut += ("[" + ticks.get(i) + ", " + counts.get(i) + "]\n"); + } + LOGGER.info(quantileOut); + + quantileOut = ""; + for (Entry e : strLengthMap.entrySet()) + quantileOut += ("length: " + e.getKey() + " has: " + e.getValue()); + LOGGER.info(quantileOut); + + br.close(); + } + + // @Test + // public void testAnsiHistogram() throws Exception { + // DTStreamingHistogram dth = new DTStreamingHistogram( + // IHistogram.FieldType.UTF8); + // dth.initialize(); + // dth.allocate(PARTITION_CARD); + // BufferedReader br = new BufferedReader(new FileReader("/Users/michael/Desktop/tpch_2_16_1/dbgen/lineitem.tbl")); + // String line = null; + // while (null != (line = br.readLine())) { + // String[] fields = line.split("\\|"); + // UTF8StringPointable key = (UTF8StringPointable) UTF8StringPointable.FACTORY.createPointable(); + // String strD = fields[COMMENT_COLUMN]; + // byte[] buf = SampleUtils.toUTF8Byte(strD.toCharArray(), 0); + // key.set(buf, 0, UTF8StringPointable.getUTFLength(buf, 0)); + // dth.addItem(key); + // } + // List> quantiles = dth.generate(); + // for (int i = 0; i < quantiles.size(); i++) { + // StringBuilder sb = new StringBuilder(); + // UTF8StringPointable.toString(sb, quantiles.get(i).getKey().getByteArray(), 0); + // System.out.print("<" + sb.toString() + ", " + quantiles.get(i).getValue() + ">\n"); + // } + // br.close(); + // dth.countReset(); + // System.out.println("Verification"); + // br = new BufferedReader(new FileReader("/Users/michael/Desktop/tpch_2_16_1/dbgen/lineitem.tbl")); + // line = null; + // while (null != (line = br.readLine())) { + // String[] fields = line.split("\\|"); + // UTF8StringPointable key = (UTF8StringPointable) UTF8StringPointable.FACTORY.createPointable(); + // String strD = fields[COMMENT_COLUMN]; + // byte[] buf = SampleUtils.toUTF8Byte(strD.toCharArray(), 0); + // key.set(buf, 0, UTF8StringPointable.getUTFLength(buf, 0)); + // dth.countItem(key); + // } + // quantiles = dth.generate(); + // for (int i = 0; i < quantiles.size(); i++) { + // StringBuilder sb = new StringBuilder(); + // UTF8StringPointable.toString(sb, quantiles.get(i).getKey().getByteArray(), 0); + // System.out.print("<" + i + ", " + sb.toString() + ", " + quantiles.get(i).getValue() + ">\n"); + // } + // } + // @Test + // public void testRandomHistogram() throws Exception { + // DTStreamingHistogram dth = new DTStreamingHistogram( + // IHistogram.FieldType.DOUBLE); + // dth.initialize(); + // dth.allocate(PARTITION_CARD); + // BufferedReader br = new BufferedReader(new FileReader("/Users/michael/Desktop/tpch_2_16_1/dbgen/customer.tbl")); + // String line = null; + // while (null != (line = br.readLine())) { + // String[] fields = line.split("\\|"); + // DoublePointable key = (DoublePointable) DoublePointable.FACTORY.createPointable(); + // byte[] buf = new byte[DoublePointable.TYPE_TRAITS.getFixedLength()]; + // key.set(buf, 0, DoublePointable.TYPE_TRAITS.getFixedLength()); + // String strD = fields[DEFAULT_COLUMN]; + // double d = Double.parseDouble(strD); + // key.setDouble(d); + // dth.addItem(key); + // } + // List> quantiles = dth.generate(); + // for (int i = 0; i < quantiles.size(); i++) { + // System.out.print("<" + quantiles.get(i).getKey().getDouble() + ", " + quantiles.get(i).getValue() + ">\n"); + // // LOGGER.warning("<" + DoublePointable.getDouble(quantiles.get(i).getKey(), 0) + ", " + // // + quantiles.get(i).getValue() + "\n"); + // } + // br.close(); + // dth.countReset(); + // System.out.println("Verification"); + // br = new BufferedReader(new FileReader("/Users/michael/Desktop/tpch_2_16_1/dbgen/customer.tbl")); + // line = null; + // while (null != (line = br.readLine())) { + // String[] fields = line.split("\\|"); + // DoublePointable key = (DoublePointable) DoublePointable.FACTORY.createPointable(); + // byte[] buf = new byte[DoublePointable.TYPE_TRAITS.getFixedLength()]; + // key.set(buf, 0, DoublePointable.TYPE_TRAITS.getFixedLength()); + // String strD = fields[DEFAULT_COLUMN]; + // double d = Double.parseDouble(strD); + // key.setDouble(d); + // dth.countItem(key); + // } + // quantiles = dth.generate(); + // for (int i = 0; i < quantiles.size(); i++) { + // System.out.print("<" + i + ", " + quantiles.get(i).getKey().getDouble() + ", " + // + quantiles.get(i).getValue() + ">\n"); + // // LOGGER.warning("<" + DoublePointable.getDouble(quantiles.get(i).getKey(), 0) + ", " + // // + quantiles.get(i).getValue() + "\n"); + // } + // } + public static > Map sortByValue(Map map) { + List> list = new LinkedList<>(map.entrySet()); + Collections.sort(list, new Comparator>() { + @Override + public int compare(Map.Entry o1, Map.Entry o2) { + return (o1.getValue()).compareTo(o2.getValue()); + } + }); + + Map result = new LinkedHashMap<>(); + for (Map.Entry entry : list) { + result.put(entry.getKey(), entry.getValue()); + } + return result; + } + + public static , V> Map sortByKey(Map map) { + List> list = new LinkedList<>(map.entrySet()); + Collections.sort(list, new Comparator>() { + @Override + public int compare(Map.Entry o1, Map.Entry o2) { + return (o1.getKey()).compareTo(o2.getKey()); + } + }); + + Map result = new LinkedHashMap<>(); + for (Map.Entry entry : list) { + result.put(entry.getKey(), entry.getValue()); + } + return result; + } + + @Test + public void testZipfanRandom() throws Exception { + String zipFanFilePath = "data/skew/zipfan.tbl"; + /*for (int part = 20; part < 1025; part *= 200) { + for (int scale = 1; scale < 3; scale++) { + PARTITION_CARD = part;*/ + DTStreamingHistogram dth = new DTStreamingHistogram( + IHistogram.FieldType.DOUBLE, true); + dth.initialize(); + dth.allocate(PARTITION_CARD, 16, true); + Map randString = new HashMap(); + BufferedReader br = new BufferedReader(new FileReader(zipFanFilePath)); + String line = null; + /*IBinaryHashFunction hashFunction = new PointableBinaryHashFunctionFactory(DoublePointable.FACTORY) + .createBinaryHashFunction();*/ + while (null != (line = br.readLine())) { + String[] fields = line.split("\t"); + DoublePointable key = (DoublePointable) DoublePointable.FACTORY.createPointable(); + byte[] buf = new byte[Double.SIZE / Byte.SIZE]; + key.set(buf, 0, Double.SIZE / Byte.SIZE); + String strD = fields[ZIPFAN_COLUMN]; + double d = Double.parseDouble(strD); + key.setDouble(d); + // randString.put(hashFunction.hash(key, 0, key.length), key); + randString.put((int) (Math.random() * 1000000000), key); + } + randString = sortByKey(randString); + + long begin = System.currentTimeMillis(); + for (Entry entry : randString.entrySet()) + dth.addItem(entry.getValue()); + List> quantiles = dth.generate(true); + String quantileOut = ""; + for (int i = 0; i < quantiles.size(); i++) { + quantileOut += ("<" + quantiles.get(i).getKey().getDouble() + ", " + quantiles.get(i).getValue() + ">\n"); + } + LOGGER.info(quantileOut); + br.close(); + dth.countReset(); + long end = System.currentTimeMillis(); + LOGGER.info("Eclipse: " + (end - begin)); + LOGGER.info("Verification"); + br = new BufferedReader(new FileReader(zipFanFilePath)); + line = null; + while (null != (line = br.readLine())) { + String[] fields = line.split("\t"); + DoublePointable key = (DoublePointable) DoublePointable.FACTORY.createPointable(); + byte[] buf = new byte[Double.SIZE / Byte.SIZE]; + key.set(buf, 0, Double.SIZE / Byte.SIZE); + String strD = fields[ZIPFAN_COLUMN]; + double d = Double.parseDouble(strD); + key.setDouble(d); + dth.countItem(key); + } + quantiles = dth.generate(true); + int maximal = 0; + int minimal = Integer.MAX_VALUE; + int total = 0; + quantileOut = ""; + for (int i = 0; i < quantiles.size(); i++) { + quantileOut += ("<" + i + ", " + quantiles.get(i).getKey().getDouble() + ", " + quantiles.get(i).getValue() + ">\n"); + int current = quantiles.get(i).getValue(); + if (current > maximal) + maximal = current; + if (current < minimal) + minimal = current; + total += current; + } + LOGGER.info(quantileOut); + } + + // public static void main(String[] args) throws IOException { + // if (args.length != 4) { + // System.out.println("Command format: command part scale equalHeight file"); + // } + // int part = Integer.parseInt(args[0]); + // int scale = Integer.parseInt(args[1]); + // boolean equalHeight = Boolean.parseBoolean(args[2]); + // String zipFanFilePath = args[3]; + // + // part = 40; + // scale = 8; + // equalHeight = true; + // zipFanFilePath = "/Users/michael/chenli/whu/algorithm/skew.txt"; + // /*for (int part = 20; part < 1025; part *= 200) { + // for (int scale = 1; scale < 3; scale++) {*/ + // Map randString = new HashMap(); + // BufferedReader br = new BufferedReader(new FileReader(zipFanFilePath)); + // String line = null; + // /*IBinaryHashFunction hashFunction = new PointableBinaryHashFunctionFactory(DoublePointable.FACTORY) + // .createBinaryHashFunction();*/ + // while (null != (line = br.readLine())) { + // String[] fields = line.split("\t"); + // DoublePointable key = (DoublePointable) DoublePointable.FACTORY.createPointable(); + // byte[] buf = new byte[Double.SIZE / Byte.SIZE]; + // key.set(buf, 0, Double.SIZE / Byte.SIZE); + // String strD = fields[ZIPFAN_COLUMN]; + // double d = Double.parseDouble(strD); + // key.setDouble(d); + // // randString.put(hashFunction.hash(key, 0, key.length), key); + // randString.put((int) (Math.random() * 1000000000), key); + // } + // randString = sortByKey(randString); + // { + // DTStreamingHistogram dth = new DTStreamingHistogram( + // IHistogram.FieldType.DOUBLE, true); + // dth.initialize(); + // dth.allocate(part, scale, equalHeight); + // System.out.println("begin sample"); + // long begin = System.currentTimeMillis(); + // int ii = 0; + // for (Entry entry : randString.entrySet()) { + // dth.addItem(entry.getValue()); + // } + // List> quantiles = dth.generate(); + // for (int i = 0; i < quantiles.size(); i++) { + // System.out.print("<" + quantiles.get(i).getKey().getDouble() + ", " + quantiles.get(i).getValue() + // + ">\n"); + // } + // br.close(); + // dth.countReset(); + // long end = System.currentTimeMillis(); + // /*System.out.println("Verification");*/ + // br = new BufferedReader(new FileReader(zipFanFilePath)); + // line = null; + // while (null != (line = br.readLine())) { + // String[] fields = line.split("\t"); + // DoublePointable key = (DoublePointable) DoublePointable.FACTORY.createPointable(); + // byte[] buf = new byte[Double.SIZE / Byte.SIZE]; + // key.set(buf, 0, Double.SIZE / Byte.SIZE); + // String strD = fields[ZIPFAN_COLUMN]; + // double d = Double.parseDouble(strD); + // key.setDouble(d); + // dth.countItem(key); + // } + // quantiles = dth.generate(); + // int maximal = 0; + // int minimal = Integer.MAX_VALUE; + // int total = 0; + // for (int i = 0; i < quantiles.size(); i++) { + // /*System.out.print("<" + i + ", " + quantiles.get(i).getKey().getDouble() + ", " + // + quantiles.get(i).getValue() + ">\n");*/ + // int current = quantiles.get(i).getValue(); + // if (current > maximal) + // maximal = current; + // if (current < minimal) + // minimal = current; + // total += current; + // } + // System.out.println(quantiles.size() + "\t" + part + "\t" + scale + "\t" + (end - begin) + "\t" + maximal + // + "\t" + minimal + "\t" + total / part + "\t" + maximal * (double) part / total + "\t" + // + dth.updateHeap); + // } + // { + // DTStreamingHistogram dth = new DTStreamingHistogram( + // IHistogram.FieldType.DOUBLE, false); + // dth.initialize(); + // dth.allocate(part, scale, equalHeight); + // System.out.println("begin sample"); + // long begin = System.currentTimeMillis(); + // for (Entry entry : randString.entrySet()) + // dth.addItem(entry.getValue()); + // List> quantiles = dth.generate(); + // for (int i = 0; i < quantiles.size(); i++) { + // System.out.print("<" + quantiles.get(i).getKey().getDouble() + ", " + quantiles.get(i).getValue() + // + ">\n"); + // } + // br.close(); + // dth.countReset(); + // long end = System.currentTimeMillis(); + // /*System.out.println("Verification");*/ + // br = new BufferedReader(new FileReader(zipFanFilePath)); + // line = null; + // while (null != (line = br.readLine())) { + // String[] fields = line.split("\t"); + // DoublePointable key = (DoublePointable) DoublePointable.FACTORY.createPointable(); + // byte[] buf = new byte[Double.SIZE / Byte.SIZE]; + // key.set(buf, 0, Double.SIZE / Byte.SIZE); + // String strD = fields[ZIPFAN_COLUMN]; + // double d = Double.parseDouble(strD); + // key.setDouble(d); + // dth.countItem(key); + // } + // quantiles = dth.generate(); + // int maximal = 0; + // int minimal = Integer.MAX_VALUE; + // int total = 0; + // for (int i = 0; i < quantiles.size(); i++) { + // /*System.out.print("<" + i + ", " + quantiles.get(i).getKey().getDouble() + ", " + // + quantiles.get(i).getValue() + ">\n");*/ + // int current = quantiles.get(i).getValue(); + // if (current > maximal) + // maximal = current; + // if (current < minimal) + // minimal = current; + // total += current; + // } + // System.out.println(quantiles.size() + "\t" + part + "\t" + scale + "\t" + (end - begin) + "\t" + maximal + // + "\t" + minimal + "\t" + total / part + "\t" + maximal * (double) part / total + "\t" + // + dth.updateHeap); + // } + // { + // DTStreamingHistogram dth = new DTStreamingHistogram( + // IHistogram.FieldType.DOUBLE, false); + // dth.initialize(); + // dth.allocate(part, scale, equalHeight); + // System.out.println("begin sample"); + // long begin = System.currentTimeMillis(); + // for (Entry entry : randString.entrySet()) + // dth.addItem(entry.getValue()); + // List> quantiles = dth.generate(); + // for (int i = 0; i < quantiles.size(); i++) { + // System.out.print("<" + quantiles.get(i).getKey().getDouble() + ", " + quantiles.get(i).getValue() + // + ">\n"); + // } + // br.close(); + // dth.countReset(); + // long end = System.currentTimeMillis(); + // /*System.out.println("Verification");*/ + // br = new BufferedReader(new FileReader(zipFanFilePath)); + // line = null; + // while (null != (line = br.readLine())) { + // String[] fields = line.split("\t"); + // DoublePointable key = (DoublePointable) DoublePointable.FACTORY.createPointable(); + // byte[] buf = new byte[Double.SIZE / Byte.SIZE]; + // key.set(buf, 0, Double.SIZE / Byte.SIZE); + // String strD = fields[ZIPFAN_COLUMN]; + // double d = Double.parseDouble(strD); + // key.setDouble(d); + // dth.countItem(key); + // } + // quantiles = dth.generate(); + // int maximal = 0; + // int minimal = Integer.MAX_VALUE; + // int total = 0; + // for (int i = 0; i < quantiles.size(); i++) { + // /*System.out.print("<" + i + ", " + quantiles.get(i).getKey().getDouble() + ", " + // + quantiles.get(i).getValue() + ">\n");*/ + // int current = quantiles.get(i).getValue(); + // if (current > maximal) + // maximal = current; + // if (current < minimal) + // minimal = current; + // total += current; + // } + // System.out.println(quantiles.size() + "\t" + part + "\t" + scale + "\t" + (end - begin) + "\t" + maximal + // + "\t" + minimal + "\t" + total / part + "\t" + maximal * (double) part / total + "\t" + // + dth.updateHeap); + // } + // /*} + // }*/ + // } + // + // @Test + // public static void testStream() throws IOException { + // int part = 3; + // int scale = 1; + // boolean equalHeight = true; + // String zipFanFilePath = "/Users/michael/chenli/whu/algorithm/stream_seq.txt"; + // /*for (int part = 20; part < 1025; part *= 200) { + // for (int scale = 1; scale < 3; scale++) {*/ + // BufferedReader br = new BufferedReader(new FileReader(zipFanFilePath)); + // String line = null; + // /*IBinaryHashFunction hashFunction = new PointableBinaryHashFunctionFactory(DoublePointable.FACTORY) + // .createBinaryHashFunction();*/ + // { + // DTStreamingHistogram dth = new DTStreamingHistogram( + // IHistogram.FieldType.DOUBLE, true); + // dth.initialize(); + // dth.allocate(part, scale, equalHeight); + // System.out.println("begin sample"); + // long begin = System.currentTimeMillis(); + // while (null != (line = br.readLine())) { + // String[] fields = line.split("\t"); + // DoublePointable key = (DoublePointable) DoublePointable.FACTORY.createPointable(); + // byte[] buf = new byte[Double.SIZE / Byte.SIZE]; + // key.set(buf, 0, Double.SIZE / Byte.SIZE); + // String strD = fields[ZIPFAN_COLUMN]; + // double d = Double.parseDouble(strD); + // key.setDouble(d); + // dth.addItem(key); + // } + // List> quantiles = dth.generate(); + // /*for (int i = 0; i < quantiles.size(); i++) { + // System.out.print("<" + quantiles.get(i).getKey().getDouble() + ", " + quantiles.get(i).getValue() + // + ">\n"); + // }*/ + // br.close(); + // dth.countReset(); + // long end = System.currentTimeMillis(); + // /*System.out.println("Verification");*/ + // br = new BufferedReader(new FileReader(zipFanFilePath)); + // line = null; + // while (null != (line = br.readLine())) { + // String[] fields = line.split("\t"); + // DoublePointable key = (DoublePointable) DoublePointable.FACTORY.createPointable(); + // byte[] buf = new byte[Double.SIZE / Byte.SIZE]; + // key.set(buf, 0, Double.SIZE / Byte.SIZE); + // String strD = fields[ZIPFAN_COLUMN]; + // double d = Double.parseDouble(strD); + // key.setDouble(d); + // dth.countItem(key); + // } + // br.close(); + // quantiles = dth.generate(); + // int maximal = 0; + // int minimal = Integer.MAX_VALUE; + // int total = 0; + // for (int i = 0; i < quantiles.size(); i++) { + // System.out.print("<" + i + ", " + quantiles.get(i).getKey().getDouble() + ", " + // + quantiles.get(i).getValue() + ">\n"); + // int current = quantiles.get(i).getValue(); + // if (current > maximal) + // maximal = current; + // if (current < minimal) + // minimal = current; + // total += current; + // } + // System.out.println(quantiles.size() + "\t" + part + "\t" + scale + "\t" + (end - begin) + "\t" + maximal + // + "\t" + minimal + "\t" + total / part + "\t" + maximal * (double) part / total + "\t" + // + dth.updateHeap); + // } + // { + // br = new BufferedReader(new FileReader(zipFanFilePath)); + // DTStreamingHistogram dth = new DTStreamingHistogram( + // IHistogram.FieldType.DOUBLE, false); + // dth.initialize(); + // dth.allocate(part, scale, equalHeight); + // System.out.println("begin sample"); + // long begin = System.currentTimeMillis(); + // while (null != (line = br.readLine())) { + // String[] fields = line.split("\t"); + // DoublePointable key = (DoublePointable) DoublePointable.FACTORY.createPointable(); + // byte[] buf = new byte[Double.SIZE / Byte.SIZE]; + // key.set(buf, 0, Double.SIZE / Byte.SIZE); + // String strD = fields[ZIPFAN_COLUMN]; + // double d = Double.parseDouble(strD); + // key.setDouble(d); + // dth.addItem(key); + // } + // List> quantiles = dth.generate(); + // /*for (int i = 0; i < quantiles.size(); i++) { + // System.out.print("<" + quantiles.get(i).getKey().getDouble() + ", " + quantiles.get(i).getValue() + // + ">\n"); + // }*/ + // br.close(); + // dth.countReset(); + // long end = System.currentTimeMillis(); + // /*System.out.println("Verification");*/ + // br = new BufferedReader(new FileReader(zipFanFilePath)); + // line = null; + // while (null != (line = br.readLine())) { + // String[] fields = line.split("\t"); + // DoublePointable key = (DoublePointable) DoublePointable.FACTORY.createPointable(); + // byte[] buf = new byte[Double.SIZE / Byte.SIZE]; + // key.set(buf, 0, Double.SIZE / Byte.SIZE); + // String strD = fields[ZIPFAN_COLUMN]; + // double d = Double.parseDouble(strD); + // key.setDouble(d); + // dth.countItem(key); + // } + // br.close(); + // quantiles = dth.generate(); + // int maximal = 0; + // int minimal = Integer.MAX_VALUE; + // int total = 0; + // for (int i = 0; i < quantiles.size(); i++) { + // System.out.print("<" + i + ", " + quantiles.get(i).getKey().getDouble() + ", " + // + quantiles.get(i).getValue() + ">\n"); + // int current = quantiles.get(i).getValue(); + // if (current > maximal) + // maximal = current; + // if (current < minimal) + // minimal = current; + // total += current; + // } + // System.out.println(quantiles.size() + "\t" + part + "\t" + scale + "\t" + (end - begin) + "\t" + maximal + // + "\t" + minimal + "\t" + total / part + "\t" + maximal * (double) part / total + "\t" + // + dth.updateHeap); + // } + // } + + // @Test + // public void testDecisionTreeZipfan() throws Exception { + // DecisionTreeHistogram dth = new DecisionTreeHistogram(); + // dth.initialize(); + // dth.allocate(300); + // BufferedReader br = new BufferedReader(new FileReader("/Users/michael/Desktop/zipfan.txt")); + // String line = null; + // while (null != (line = br.readLine())) { + // String[] fields = line.split("\t"); + // byte[] key = new byte[Double.SIZE / Byte.SIZE]; + // String strD = fields[ZIPFAN_COLUMN]; + // double d = Double.parseDouble(strD); + // DoublePointable.setDouble(key, 0, d); + // dth.addItem(key); + // } + // List> quantiles = dth.generate(); + // for (int i = 0; i < quantiles.size(); i++) { + // System.out.print("<" + DoublePointable.getDouble(quantiles.get(i).getKey(), 0) + ", " + // + quantiles.get(i).getValue() + ">\n"); + //// LOGGER.warning("<" + DoublePointable.getDouble(quantiles.get(i).getKey(), 0) + ", " + //// + quantiles.get(i).getValue() + "\n"); + // } + // br.close(); + // dth.countReset(); + // System.out.println("Verification"); + // br = new BufferedReader(new FileReader("/Users/michael/Desktop/zipfan.txt")); + // line = null; + // while (null != (line = br.readLine())) { + // String[] fields = line.split("\t"); + // byte[] key = new byte[Double.SIZE / Byte.SIZE]; + // String strD = fields[ZIPFAN_COLUMN]; + // double d = Double.parseDouble(strD); + // DoublePointable.setDouble(key, 0, d); + // dth.countItem(key); + // } + // quantiles = dth.generate(); + // for (int i = 0; i < quantiles.size(); i++) { + // System.out.print("<" + i + ", " + DoublePointable.getDouble(quantiles.get(i).getKey(), 0) + ", " + // + quantiles.get(i).getValue() + ">\n"); + //// LOGGER.warning("<" + DoublePointable.getDouble(quantiles.get(i).getKey(), 0) + ", " + //// + quantiles.get(i).getValue() + "\n"); + // } + // } + + @After + public void cleanUpStreams() { + System.setOut(null); + System.setErr(null); + } +} diff --git a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java index ee2bff5b80a..040f0a743f1 100644 --- a/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java +++ b/hyracks-fullstack/hyracks/hyracks-test-support/src/main/java/org/apache/hyracks/test/support/TestTaskContext.java @@ -35,6 +35,7 @@ import org.apache.hyracks.api.io.IIOManager; import org.apache.hyracks.api.job.profiling.counters.ICounterContext; import org.apache.hyracks.api.resources.IDeallocatable; +import org.apache.hyracks.control.nc.Task; import org.apache.hyracks.control.nc.io.IOManager; import org.apache.hyracks.control.nc.io.WorkspaceFileFactory; @@ -155,4 +156,13 @@ public Object getSharedObject() { @Override public void setSharedObject(Object sharedObject) { } + + @Override + public void setGlobalState(int partition, final IStateObject state) { + } + + @Override + public IStateObject getGlobalState(int partition) { + return null; + } }