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; + } }