You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hivemall.apache.org by my...@apache.org on 2017/01/16 07:03:17 UTC

[1/6] incubator-hivemall git commit: Close #13: Implement Kernel Expansion Passive Aggressive Classification

Repository: incubator-hivemall
Updated Branches:
  refs/heads/master 273851f39 -> 391e7f1c6


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/resources/ddl/define-all-as-permanent.hive
----------------------------------------------------------------------
diff --git a/resources/ddl/define-all-as-permanent.hive b/resources/ddl/define-all-as-permanent.hive
index 72835b1..fe7b6f7 100644
--- a/resources/ddl/define-all-as-permanent.hive
+++ b/resources/ddl/define-all-as-permanent.hive
@@ -43,6 +43,12 @@ CREATE FUNCTION train_scw2 as 'hivemall.classifier.SoftConfideceWeightedUDTF$SCW
 DROP FUNCTION IF EXISTS train_adagrad_rda;
 CREATE FUNCTION train_adagrad_rda as 'hivemall.classifier.AdaGradRDAUDTF' USING JAR '${hivemall_jar}';
 
+DROP FUNCTION IF EXISTS train_kpa;
+CREATE FUNCTION train_kpa as 'hivemall.classifier.KernelExpansionPassiveAggressiveUDTF' USING JAR '${hivemall_jar}';
+
+DROP FUNCTION IF EXISTS kpa_predict;
+CREATE FUNCTION kpa_predict as 'hivemall.classifier.KPAPredictUDAF' USING JAR '${hivemall_jar}';
+
 --------------------------------
 --  Multiclass classification --
 -------------------------------- 
@@ -186,6 +192,9 @@ CREATE FUNCTION polynomial_features as 'hivemall.ftvec.pairing.PolynomialFeature
 DROP FUNCTION IF EXISTS powered_features;
 CREATE FUNCTION powered_features as 'hivemall.ftvec.pairing.PoweredFeaturesUDF' USING JAR '${hivemall_jar}';
 
+DROP FUNCTION IF EXISTS feature_pairs;
+CREATE FUNCTION feature_pairs as 'hivemall.ftvec.pairing.FeaturePairsUDTF' USING JAR '${hivemall_jar}';
+
 -----------------------
 -- scaling functions --
 -----------------------

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/resources/ddl/define-all.hive
----------------------------------------------------------------------
diff --git a/resources/ddl/define-all.hive b/resources/ddl/define-all.hive
index 351303e..b0a0226 100644
--- a/resources/ddl/define-all.hive
+++ b/resources/ddl/define-all.hive
@@ -39,6 +39,12 @@ create temporary function train_scw2 as 'hivemall.classifier.SoftConfideceWeight
 drop temporary function train_adagrad_rda;
 create temporary function train_adagrad_rda as 'hivemall.classifier.AdaGradRDAUDTF';
 
+drop temporary function train_kpa;
+create temporary function train_kpa as 'hivemall.classifier.KernelExpansionPassiveAggressiveUDTF';
+
+drop temporary function kpa_predict;
+create temporary function kpa_predict as 'hivemall.classifier.KPAPredictUDAF';
+
 --------------------------------
 --  Multiclass classification --
 -------------------------------- 
@@ -182,6 +188,9 @@ create temporary function polynomial_features as 'hivemall.ftvec.pairing.Polynom
 drop temporary function powered_features;
 create temporary function powered_features as 'hivemall.ftvec.pairing.PoweredFeaturesUDF';
 
+drop temporary function feature_pairs;
+create temporary function feature_pairs as 'hivemall.ftvec.pairing.FeaturePairsUDTF';
+
 -----------------------
 -- scaling functions --
 -----------------------


[6/6] incubator-hivemall git commit: Close #13: Implement Kernel Expansion Passive Aggressive Classification

Posted by my...@apache.org.
Close #13: Implement Kernel Expansion Passive Aggressive Classification


Project: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/commit/391e7f1c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/tree/391e7f1c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-hivemall/diff/391e7f1c

Branch: refs/heads/master
Commit: 391e7f1c65c7084acfbfe6b2491765a4dcd8212a
Parents: 273851f
Author: Sotaro Sugimoto <so...@gmail.com>
Authored: Mon Jan 16 16:01:46 2017 +0900
Committer: myui <yu...@gmail.com>
Committed: Mon Jan 16 16:01:46 2017 +0900

----------------------------------------------------------------------
 .../src/main/java/hivemall/LearnerBaseUDTF.java |     4 +-
 .../anomaly/SingularSpectrumTransformUDF.java   |    22 +-
 .../classifier/BinaryOnlineClassifierUDTF.java  |    11 +-
 .../hivemall/classifier/KPAPredictUDAF.java     |   228 +
 .../KernelExpansionPassiveAggressiveUDTF.java   |   378 +
 .../hivemall/ensemble/bagging/VotedAvgUDAF.java |     8 +-
 .../ensemble/bagging/WeightVotedAvgUDAF.java    |     8 +-
 .../ftvec/pairing/FeaturePairsUDTF.java         |   232 +
 .../main/java/hivemall/model/FeatureValue.java  |    29 +-
 .../utils/collections/FloatArrayList.java       |   152 +
 .../collections/Int2FloatOpenHashTable.java     |     3 +
 .../java/hivemall/utils/hadoop/HiveUtils.java   |    51 +
 .../hivemall/utils/hashing/HashFunction.java    |    14 +
 .../java/hivemall/utils/lang/Preconditions.java |    18 +-
 .../java/hivemall/utils/math/MathUtils.java     |    18 +-
 .../hivemall/anomaly/ChangeFinder1DTest.java    |     2 +-
 .../hivemall/anomaly/ChangeFinder2DTest.java    |     2 +-
 ...ernelExpansionPassiveAggressiveUDTFTest.java |   158 +
 .../fm/FactorizationMachineUDTFTest.java        |    16 +-
 .../FieldAwareFactorizationMachineUDTFTest.java |    16 +-
 .../mf/BPRMatrixFactorizationUDTFTest.java      |    16 +-
 .../collections/Int2FloatOpenHashMapTest.java   |    13 +
 .../utils/io/Base91OutputStreamTest.java        |     2 +-
 .../hivemall/utils/lang/PreconditionsTest.java  |     2 +-
 .../test/resources/hivemall/anomaly/cf1d.csv    |  2503 -
 .../test/resources/hivemall/anomaly/cf1d.csv.gz |   Bin 0 -> 17417 bytes
 .../hivemall/classifier/news20-small.binary.gz  |   Bin 0 -> 121787 bytes
 core/src/test/resources/hivemall/fm/5107786.txt |   200 -
 .../test/resources/hivemall/fm/5107786.txt.gz   |   Bin 0 -> 2712 bytes
 .../test/resources/hivemall/fm/bigdata.tr.txt   |   200 -
 .../resources/hivemall/fm/bigdata.tr.txt.gz     |   Bin 0 -> 4544 bytes
 core/src/test/resources/hivemall/mf/ml1k.test   | 20000 -----
 .../src/test/resources/hivemall/mf/ml1k.test.gz |   Bin 0 -> 78568 bytes
 core/src/test/resources/hivemall/mf/ml1k.train  | 80000 -----------------
 .../test/resources/hivemall/mf/ml1k.train.gz    |   Bin 0 -> 311183 bytes
 resources/ddl/define-all-as-permanent.hive      |     9 +
 resources/ddl/define-all.hive                   |     9 +
 37 files changed, 1367 insertions(+), 102957 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/main/java/hivemall/LearnerBaseUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/LearnerBaseUDTF.java b/core/src/main/java/hivemall/LearnerBaseUDTF.java
index f1ad99e..17c3ebc 100644
--- a/core/src/main/java/hivemall/LearnerBaseUDTF.java
+++ b/core/src/main/java/hivemall/LearnerBaseUDTF.java
@@ -161,11 +161,13 @@ public abstract class LearnerBaseUDTF extends UDTFWithOptions {
         return cl;
     }
 
+    @Nullable
     protected PredictionModel createModel() {
         return createModel(null);
     }
 
-    protected PredictionModel createModel(String label) {
+    @Nonnull
+    protected PredictionModel createModel(@Nullable String label) {
         PredictionModel model;
         final boolean useCovar = useCovariance();
         if (dense_model) {

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/main/java/hivemall/anomaly/SingularSpectrumTransformUDF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/anomaly/SingularSpectrumTransformUDF.java b/core/src/main/java/hivemall/anomaly/SingularSpectrumTransformUDF.java
index d699a95..1fac3e7 100644
--- a/core/src/main/java/hivemall/anomaly/SingularSpectrumTransformUDF.java
+++ b/core/src/main/java/hivemall/anomaly/SingularSpectrumTransformUDF.java
@@ -125,19 +125,17 @@ public final class SingularSpectrumTransformUDF extends UDFWithOptions {
         this._params.changepointThreshold = Primitives.parseDouble(cl.getOptionValue("th"),
             _params.changepointThreshold);
 
-        Preconditions.checkArgument(_params.w >= 2, "w must be greather than 1: " + _params.w,
-            UDFArgumentException.class);
-        Preconditions.checkArgument(_params.r >= 1, "r must be greater than 0: " + _params.r,
-            UDFArgumentException.class);
-        Preconditions.checkArgument(_params.k >= 1, "k must be greater than 0: " + _params.k,
-            UDFArgumentException.class);
-        Preconditions.checkArgument(_params.k >= _params.r,
-            "k must be equals to or greather than r: k=" + _params.k + ", r" + _params.r,
-            UDFArgumentException.class);
+        Preconditions.checkArgument(_params.w >= 2, UDFArgumentException.class,
+            "w must be greather than 1: " + _params.w);
+        Preconditions.checkArgument(_params.r >= 1, UDFArgumentException.class,
+            "r must be greater than 0: " + _params.r);
+        Preconditions.checkArgument(_params.k >= 1, UDFArgumentException.class,
+            "k must be greater than 0: " + _params.k);
+        Preconditions.checkArgument(_params.k >= _params.r, UDFArgumentException.class,
+            "k must be equals to or greather than r: k=" + _params.k + ", r" + _params.r);
         Preconditions.checkArgument(_params.changepointThreshold > 0.d
-                && _params.changepointThreshold < 1.d,
-            "changepointThreshold must be in range (0, 1): " + _params.changepointThreshold,
-            UDFArgumentException.class);
+                && _params.changepointThreshold < 1.d, UDFArgumentException.class,
+            "changepointThreshold must be in range (0, 1): " + _params.changepointThreshold);
 
         return cl;
     }

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/main/java/hivemall/classifier/BinaryOnlineClassifierUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/classifier/BinaryOnlineClassifierUDTF.java b/core/src/main/java/hivemall/classifier/BinaryOnlineClassifierUDTF.java
index c9274e4..b0e2efd 100644
--- a/core/src/main/java/hivemall/classifier/BinaryOnlineClassifierUDTF.java
+++ b/core/src/main/java/hivemall/classifier/BinaryOnlineClassifierUDTF.java
@@ -51,8 +51,8 @@ import org.apache.hadoop.io.FloatWritable;
 public abstract class BinaryOnlineClassifierUDTF extends LearnerBaseUDTF {
     private static final Log logger = LogFactory.getLog(BinaryOnlineClassifierUDTF.class);
 
-    private ListObjectInspector featureListOI;
-    private PrimitiveObjectInspector labelOI;
+    protected ListObjectInspector featureListOI;
+    protected PrimitiveObjectInspector labelOI;
     private boolean parseFeature;
 
     protected PredictionModel model;
@@ -122,7 +122,7 @@ public abstract class BinaryOnlineClassifierUDTF extends LearnerBaseUDTF {
     }
 
     @Nullable
-    protected final FeatureValue[] parseFeatures(@Nonnull final List<?> features) {
+    FeatureValue[] parseFeatures(@Nonnull final List<?> features) {
         final int size = features.size();
         if (size == 0) {
             return null;
@@ -151,6 +151,7 @@ public abstract class BinaryOnlineClassifierUDTF extends LearnerBaseUDTF {
         assert (label == -1 || label == 0 || label == 1) : label;
     }
 
+    //@VisibleForTesting
     void train(List<?> features, int label) {
         FeatureValue[] featureVector = parseFeatures(features);
         train(featureVector, label);
@@ -166,7 +167,7 @@ public abstract class BinaryOnlineClassifierUDTF extends LearnerBaseUDTF {
         }
     }
 
-    protected float predict(@Nonnull final FeatureValue[] features) {
+    float predict(@Nonnull final FeatureValue[] features) {
         float score = 0.f;
         for (FeatureValue f : features) {// a += w[i] * x[i]
             if (f == null) {
@@ -247,7 +248,7 @@ public abstract class BinaryOnlineClassifierUDTF extends LearnerBaseUDTF {
     }
 
     @Override
-    public final void close() throws HiveException {
+    public void close() throws HiveException {
         super.close();
         if (model != null) {
             int numForwarded = 0;

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/main/java/hivemall/classifier/KPAPredictUDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/classifier/KPAPredictUDAF.java b/core/src/main/java/hivemall/classifier/KPAPredictUDAF.java
new file mode 100644
index 0000000..72409d9
--- /dev/null
+++ b/core/src/main/java/hivemall/classifier/KPAPredictUDAF.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 hivemall.classifier;
+
+import hivemall.utils.hadoop.HiveUtils;
+import hivemall.utils.lang.Preconditions;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.udf.generic.AbstractGenericUDAFResolver;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.AbstractAggregationBuffer;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator.AggregationType;
+import org.apache.hadoop.hive.ql.util.JavaDataModel;
+import org.apache.hadoop.hive.serde2.io.DoubleWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+
+@Description(
+        name = "kpa_predict",
+        value = "_FUNC_(@Nonnull double xh, @Nonnull double xk, @Nullable float w0, @Nonnull float w1, @Nonnull float w2, @Nullable float w3)"
+                + " - Returns a prediction value in Double")
+public final class KPAPredictUDAF extends AbstractGenericUDAFResolver {
+
+    @Override
+    public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters) throws SemanticException {
+        if (parameters.length != 6) {
+            throw new UDFArgumentException(
+                "_FUNC_(double xh, double xk, float w0, float w1, float w2, float w3) takes exactly 6 arguments but got: "
+                        + parameters.length);
+        }
+        if (!HiveUtils.isNumberTypeInfo(parameters[0])) {
+            throw new UDFArgumentTypeException(0, "Number type is expected for xh (1st argument): "
+                    + parameters[0].getTypeName());
+        }
+        if (!HiveUtils.isNumberTypeInfo(parameters[1])) {
+            throw new UDFArgumentTypeException(1, "Number type is expected for xk (2nd argument): "
+                    + parameters[1].getTypeName());
+        }
+        if (!HiveUtils.isNumberTypeInfo(parameters[2])) {
+            throw new UDFArgumentTypeException(2, "Number type is expected for w0 (3rd argument): "
+                    + parameters[2].getTypeName());
+        }
+        if (!HiveUtils.isNumberTypeInfo(parameters[3])) {
+            throw new UDFArgumentTypeException(3, "Number type is expected for w1 (4th argument): "
+                    + parameters[3].getTypeName());
+        }
+        if (!HiveUtils.isNumberTypeInfo(parameters[4])) {
+            throw new UDFArgumentTypeException(4, "Number type is expected for w2 (5th argument): "
+                    + parameters[4].getTypeName());
+        }
+        if (!HiveUtils.isNumberTypeInfo(parameters[5])) {
+            throw new UDFArgumentTypeException(5, "Number type is expected for w3 (6th argument): "
+                    + parameters[5].getTypeName());
+        }
+
+        return new Evaluator();
+    }
+
+    public static class Evaluator extends GenericUDAFEvaluator {
+
+        @Nullable
+        private transient PrimitiveObjectInspector xhOI, xkOI;
+        @Nullable
+        private transient PrimitiveObjectInspector w0OI, w1OI, w2OI, w3OI;
+
+        public Evaluator() {}
+
+        @Override
+        public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException {
+            super.init(m, parameters);
+
+            // initialize input
+            if (m == Mode.PARTIAL1 || m == Mode.COMPLETE) {// from original data
+                this.xhOI = HiveUtils.asNumberOI(parameters[0]);
+                this.xkOI = HiveUtils.asNumberOI(parameters[1]);
+                this.w0OI = HiveUtils.asNumberOI(parameters[2]);
+                this.w1OI = HiveUtils.asNumberOI(parameters[3]);
+                this.w2OI = HiveUtils.asNumberOI(parameters[4]);
+                this.w3OI = HiveUtils.asNumberOI(parameters[5]);
+            }
+
+            return PrimitiveObjectInspectorFactory.writableDoubleObjectInspector;
+        }
+
+        @Override
+        public AggrBuffer getNewAggregationBuffer() throws HiveException {
+            return new AggrBuffer();
+        }
+
+        @Override
+        public void reset(@SuppressWarnings("deprecation") AggregationBuffer agg)
+                throws HiveException {
+            AggrBuffer aggr = (AggrBuffer) agg;
+            aggr.reset();
+        }
+
+        @Override
+        public void iterate(@SuppressWarnings("deprecation") AggregationBuffer agg,
+                Object[] parameters) throws HiveException {
+            Preconditions.checkArgument(parameters.length == 6, HiveException.class);
+
+            final AggrBuffer aggr = (AggrBuffer) agg;
+
+            if (parameters[0] /* xh */!= null) {
+                double xh = HiveUtils.getDouble(parameters[0], xhOI);
+                if (parameters[1] /* xk */!= null) {
+                    if (parameters[5] /* w3hk */== null) {
+                        return;
+                    }
+                    // xh, xk, w3hk
+                    double xk = HiveUtils.getDouble(parameters[1], xkOI);
+                    double w3hk = HiveUtils.getDouble(parameters[5], w3OI);
+                    aggr.addW3(xh, xk, w3hk);
+                } else {
+                    if (parameters[3] /* w1h */== null) {
+                        return;
+                    }
+                    // xh, w1h, w2h
+                    Preconditions.checkNotNull(parameters[4], HiveException.class);
+                    double w1h = HiveUtils.getDouble(parameters[3], w1OI);
+                    double w2h = HiveUtils.getDouble(parameters[4], w2OI);
+                    aggr.addW1W2(xh, w1h, w2h);
+                }
+            } else if (parameters[2] /* w0 */!= null) {
+                // w0
+                double w0 = HiveUtils.getDouble(parameters[2], w0OI);
+                aggr.addW0(w0);
+            }
+        }
+
+        @Override
+        public Object terminatePartial(@SuppressWarnings("deprecation") AggregationBuffer agg)
+                throws HiveException {
+            AggrBuffer aggr = (AggrBuffer) agg;
+            double v = aggr.get();
+            return new DoubleWritable(v);
+        }
+
+        @Override
+        public void merge(@SuppressWarnings("deprecation") AggregationBuffer agg, Object partial)
+                throws HiveException {
+            if (partial == null) {
+                return;
+            }
+
+            AggrBuffer aggr = (AggrBuffer) agg;
+            DoubleWritable other = (DoubleWritable) partial;
+            double v = other.get();
+            aggr.merge(v);
+        }
+
+        @Override
+        public DoubleWritable terminate(@SuppressWarnings("deprecation") AggregationBuffer agg)
+                throws HiveException {
+            AggrBuffer aggr = (AggrBuffer) agg;
+            double v = aggr.get();
+            return new DoubleWritable(v);
+        }
+
+    }
+
+    @AggregationType(estimable = true)
+    static class AggrBuffer extends AbstractAggregationBuffer {
+
+        double score;
+
+        AggrBuffer() {
+            super();
+            reset();
+        }
+
+        @Override
+        public int estimate() {
+            return JavaDataModel.PRIMITIVES2;
+        }
+
+        void reset() {
+            this.score = 0.d;
+        }
+
+        double get() {
+            return score;
+        }
+
+        void addW0(@Nonnull double w0) {
+            this.score += w0;
+        }
+
+        void addW1W2(final double xh, final double w1h, final double w2h) {
+            this.score += (w1h * xh + w2h * xh * xh);
+        }
+
+        void addW3(final double xh, final double xk, final double w3hk) {
+            this.score += (w3hk * xh * xk);
+        }
+
+        void merge(final double other) {
+            this.score += other;
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/main/java/hivemall/classifier/KernelExpansionPassiveAggressiveUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/classifier/KernelExpansionPassiveAggressiveUDTF.java b/core/src/main/java/hivemall/classifier/KernelExpansionPassiveAggressiveUDTF.java
new file mode 100644
index 0000000..3e28932
--- /dev/null
+++ b/core/src/main/java/hivemall/classifier/KernelExpansionPassiveAggressiveUDTF.java
@@ -0,0 +1,378 @@
+/*
+ * 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 hivemall.classifier;
+
+import hivemall.common.LossFunctions;
+import hivemall.model.FeatureValue;
+import hivemall.model.PredictionModel;
+import hivemall.model.PredictionResult;
+import hivemall.utils.collections.Int2FloatOpenHashTable;
+import hivemall.utils.collections.Int2FloatOpenHashTable.IMapIterator;
+import hivemall.utils.hashing.HashFunction;
+import hivemall.utils.lang.Preconditions;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.io.FloatWritable;
+import org.apache.hadoop.io.IntWritable;
+
+/**
+ * Degree-2 polynomial kernel expansion Passive Aggressive.
+ * 
+ * <pre>
+ * Hideki Isozaki and Hideto Kazawa: Efficient Support Vector Classifiers for Named Entity Recognition, Proc.COLING, 2002
+ * </pre>
+ */
+@Description(name = "train_kpa",
+        value = "_FUNC_(array<string|int|bigint> features, int label [, const string options])"
+                + " - returns a relation <h int, hk int, float w0, float w1, float w2, float w3>")
+public final class KernelExpansionPassiveAggressiveUDTF extends BinaryOnlineClassifierUDTF {
+
+    // ------------------------------------
+    // Hyper parameters
+    private float _pkc;
+    // Algorithm
+    private Algorithm _algo;
+
+    // ------------------------------------
+    // Model parameters
+
+    private float _w0;
+    private Int2FloatOpenHashTable _w1;
+    private Int2FloatOpenHashTable _w2;
+    private Int2FloatOpenHashTable _w3;
+
+    // ------------------------------------
+
+    private float _loss;
+
+    public KernelExpansionPassiveAggressiveUDTF() {}
+
+    //@VisibleForTesting
+    float getLoss() {//only used for testing purposes at the moment
+        return _loss;
+    }
+
+    @Override
+    protected Options getOptions() {
+        Options opts = new Options();
+        opts.addOption("pkc", true,
+            "Constant c inside polynomial kernel K = (dot(xi,xj) + c)^2 [default 1.0]");
+        opts.addOption("algo", "algorithm", true,
+            "Algorithm for calculating loss [pa, pa1 (default), pa2]");
+        opts.addOption("c", "aggressiveness", true,
+            "Aggressiveness parameter C for PA-1 and PA-2 [default 1.0]");
+        return opts;
+    }
+
+    @Override
+    protected CommandLine processOptions(ObjectInspector[] argOIs) throws UDFArgumentException {
+        float pkc = 1.f;
+        float c = 1.f;
+        String algo = "pa1";
+
+        final CommandLine cl = super.processOptions(argOIs);
+        if (cl != null) {
+            String pkc_str = cl.getOptionValue("pkc");
+            if (pkc_str != null) {
+                pkc = Float.parseFloat(pkc_str);
+            }
+            String c_str = cl.getOptionValue("c");
+            if (c_str != null) {
+                c = Float.parseFloat(c_str);
+                if (c <= 0.f) {
+                    throw new UDFArgumentException("Aggressiveness parameter C must be C > 0: " + c);
+                }
+            }
+            algo = cl.getOptionValue("algo", algo);
+        }
+
+        if ("pa1".equalsIgnoreCase(algo)) {
+            this._algo = new PA1(c);
+        } else if ("pa2".equalsIgnoreCase(algo)) {
+            this._algo = new PA2(c);
+        } else if ("pa".equalsIgnoreCase(algo)) {
+            this._algo = new PA();
+        } else {
+            throw new UDFArgumentException("Unsupported algorithm: " + algo);
+        }
+        this._pkc = pkc;
+
+        return cl;
+    }
+
+    interface Algorithm {
+        float eta(final float loss, @Nonnull final PredictionResult margin);
+    }
+
+    static class PA implements Algorithm {
+
+        PA() {}
+
+        @Override
+        public float eta(float loss, PredictionResult margin) {
+            return loss / margin.getSquaredNorm();
+        }
+    }
+
+    static class PA1 implements Algorithm {
+        private final float c;
+
+        PA1(float c) {
+            this.c = c;
+        }
+
+        @Override
+        public float eta(float loss, PredictionResult margin) {
+            float squared_norm = margin.getSquaredNorm();
+            float eta = loss / squared_norm;
+            return Math.min(c, eta);
+        }
+    }
+
+    static class PA2 implements Algorithm {
+        private final float c;
+
+        PA2(float c) {
+            this.c = c;
+        }
+
+        @Override
+        public float eta(float loss, PredictionResult margin) {
+            float squared_norm = margin.getSquaredNorm();
+            float eta = loss / (squared_norm + (0.5f / c));
+            return eta;
+        }
+    }
+
+    @Override
+    protected PredictionModel createModel() {
+        this._w0 = 0.f;
+        this._w1 = new Int2FloatOpenHashTable(16384);
+        _w1.defaultReturnValue(0.f);
+        this._w2 = new Int2FloatOpenHashTable(16384);
+        _w2.defaultReturnValue(0.f);
+        this._w3 = new Int2FloatOpenHashTable(16384);
+        _w3.defaultReturnValue(0.f);
+
+        return null;
+    }
+
+    @Override
+    protected StructObjectInspector getReturnOI(ObjectInspector featureRawOI) {
+        ArrayList<String> fieldNames = new ArrayList<String>();
+        ArrayList<ObjectInspector> fieldOIs = new ArrayList<ObjectInspector>();
+
+        fieldNames.add("h");
+        fieldOIs.add(PrimitiveObjectInspectorFactory.writableIntObjectInspector);
+        fieldNames.add("w0");
+        fieldOIs.add(PrimitiveObjectInspectorFactory.writableFloatObjectInspector);
+        fieldNames.add("w1");
+        fieldOIs.add(PrimitiveObjectInspectorFactory.writableFloatObjectInspector);
+        fieldNames.add("w2");
+        fieldOIs.add(PrimitiveObjectInspectorFactory.writableFloatObjectInspector);
+        fieldNames.add("hk");
+        fieldOIs.add(PrimitiveObjectInspectorFactory.writableIntObjectInspector);
+        fieldNames.add("w3");
+        fieldOIs.add(PrimitiveObjectInspectorFactory.writableFloatObjectInspector);
+
+        return ObjectInspectorFactory.getStandardStructObjectInspector(fieldNames, fieldOIs);
+    }
+
+    @Nullable
+    FeatureValue[] parseFeatures(@Nonnull final List<?> features) {
+        final int size = features.size();
+        if (size == 0) {
+            return null;
+        }
+
+        final FeatureValue[] featureVector = new FeatureValue[size];
+        for (int i = 0; i < size; i++) {
+            Object f = features.get(i);
+            if (f == null) {
+                continue;
+            }
+            FeatureValue fv = FeatureValue.parse(f, true);
+            featureVector[i] = fv;
+        }
+        return featureVector;
+    }
+
+    @Override
+    protected void train(@Nonnull final FeatureValue[] features, final int label) {
+        final float y = label > 0 ? 1.f : -1.f;
+
+        PredictionResult margin = calcScoreWithKernelAndNorm(features);
+        float p = margin.getScore();
+        float loss = LossFunctions.hingeLoss(p, y); // 1.0 - y * p
+        this._loss = loss;
+
+        if (loss > 0.f) { // y * p < 1
+            updateKernel(y, loss, margin, features);
+        }
+    }
+
+    @Override
+    float predict(@Nonnull final FeatureValue[] features) {
+        float score = 0.f;
+
+        for (int i = 0; i < features.length; ++i) {
+            if (features[i] == null) {
+                continue;
+            }
+            int h = features[i].getFeatureAsInt();
+            float w1 = _w1.get(h);
+            float w2 = _w2.get(h);
+            double xi = features[i].getValue();
+            double xx = xi * xi;
+            score += w1 * xi;
+            score += w2 * xx;
+            for (int j = i + 1; j < features.length; ++j) {
+                int k = features[j].getFeatureAsInt();
+                int hk = HashFunction.hash(h, k, true);
+                float w3 = _w3.get(hk);
+                double xj = features[j].getValue();
+                score += xi * xj * w3;
+            }
+        }
+
+        return score;
+    }
+
+    @Nonnull
+    final PredictionResult calcScoreWithKernelAndNorm(@Nonnull final FeatureValue[] features) {
+        float score = _w0;
+        float norm = 0.f;
+        for (int i = 0; i < features.length; ++i) {
+            if (features[i] == null) {
+                continue;
+            }
+            int h = features[i].getFeatureAsInt();
+            float w1 = _w1.get(h);
+            float w2 = _w2.get(h);
+            double xi = features[i].getValue();
+            double xx = xi * xi;
+            score += w1 * xi;
+            score += w2 * xx;
+            norm += xx;
+            for (int j = i + 1; j < features.length; ++j) {
+                int k = features[j].getFeatureAsInt();
+                int hk = HashFunction.hash(h, k, true);
+                float w3 = _w3.get(hk);
+                double xj = features[j].getValue();
+                score += xi * xj * w3;
+            }
+        }
+        return new PredictionResult(score).squaredNorm(norm);
+    }
+
+    protected void updateKernel(final float label, final float loss,
+            @Nonnull final PredictionResult margin, @Nonnull final FeatureValue[] features) {
+        float eta = _algo.eta(loss, margin);
+        float coeff = eta * label;
+        expandKernel(features, coeff);
+    }
+
+    private void expandKernel(@Nonnull final FeatureValue[] supportVector, final float alpha) {
+        final float pkc = _pkc;
+        // W0 += \u03b1 c^2
+        this._w0 += alpha * pkc * pkc;
+
+        for (int i = 0; i < supportVector.length; ++i) {
+            final FeatureValue si = supportVector[i];
+            final int h = si.getFeatureAsInt();
+            float Zih = si.getValueAsFloat();
+
+            float alphaZih = alpha * Zih;
+            final float alphaZih2 = alphaZih * 2.f;
+
+            // W1[h] += 2 c \u03b1 Zi[h]
+            _w1.put(h, _w1.get(h) + pkc * alphaZih2);
+            // W2[h] += \u03b1 Zi[h]^2
+            _w2.put(h, _w2.get(h) + alphaZih * Zih);
+
+            for (int j = i + 1; j < supportVector.length; ++j) {
+                FeatureValue sj = supportVector[j];
+                int k = sj.getFeatureAsInt();
+                int hk = HashFunction.hash(h, k, true);
+                float Zjk = sj.getValueAsFloat();
+
+                // W3 += 2 \u03b1 Zi[h] Zi[k]
+                _w3.put(hk, _w3.get(hk) + alphaZih2 * Zjk);
+            }
+        }
+    }
+
+    @Override
+    public void close() throws HiveException {
+        final IntWritable h = new IntWritable(0); // row[0]
+        final FloatWritable w0 = new FloatWritable(_w0); // row[1]
+        final FloatWritable w1 = new FloatWritable(); // row[2]
+        final FloatWritable w2 = new FloatWritable(); // row[3]
+        final IntWritable hk = new IntWritable(0); // row[4]
+        final FloatWritable w3 = new FloatWritable(); // row[5]
+        final Object[] row = new Object[] {h, w0, null, null, null, null};
+        forward(row); // 0(f), w0
+        row[1] = null;
+
+        row[2] = w1;
+        row[3] = w2;
+        final Int2FloatOpenHashTable w2map = _w2;
+        final IMapIterator w1itor = _w1.entries();
+        while (w1itor.next() != -1) {
+            int k = w1itor.getKey();
+            Preconditions.checkArgument(k > 0, HiveException.class);
+            h.set(k);
+            w1.set(w1itor.getValue());
+            w2.set(w2map.get(k));
+            forward(row); // h(f), w1, w2
+        }
+        this._w1 = null;
+        this._w2 = null;
+
+        row[0] = null;
+        row[2] = null;
+        row[3] = null;
+        row[4] = hk;
+        row[5] = w3;
+        final IMapIterator w3itor = _w3.entries();
+        while (w3itor.next() != -1) {
+            int k = w3itor.getKey();
+            Preconditions.checkArgument(k > 0, HiveException.class);
+            hk.set(k);
+            w3.set(w3itor.getValue());
+            forward(row); // hk(f), w3
+        }
+        this._w3 = null;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/main/java/hivemall/ensemble/bagging/VotedAvgUDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ensemble/bagging/VotedAvgUDAF.java b/core/src/main/java/hivemall/ensemble/bagging/VotedAvgUDAF.java
index 8b40142..a9bcab1 100644
--- a/core/src/main/java/hivemall/ensemble/bagging/VotedAvgUDAF.java
+++ b/core/src/main/java/hivemall/ensemble/bagging/VotedAvgUDAF.java
@@ -18,6 +18,8 @@
  */
 package hivemall.ensemble.bagging;
 
+import javax.annotation.Nullable;
+
 import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.UDAF;
 import org.apache.hadoop.hive.ql.exec.UDAFEvaluator;
@@ -53,11 +55,15 @@ public final class VotedAvgUDAF extends UDAF {
             this.partial = null;
         }
 
-        public boolean iterate(double w) {
+        public boolean iterate(@Nullable DoubleWritable o) {
+            if (o == null) {
+                return true;
+            }
             if (partial == null) {
                 this.partial = new PartialResult();
                 partial.init();
             }
+            double w = o.get();
             if (w > 0) {
                 partial.positiveSum += w;
                 partial.positiveCnt++;

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/main/java/hivemall/ensemble/bagging/WeightVotedAvgUDAF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ensemble/bagging/WeightVotedAvgUDAF.java b/core/src/main/java/hivemall/ensemble/bagging/WeightVotedAvgUDAF.java
index a7d63be..4e7ea1b 100644
--- a/core/src/main/java/hivemall/ensemble/bagging/WeightVotedAvgUDAF.java
+++ b/core/src/main/java/hivemall/ensemble/bagging/WeightVotedAvgUDAF.java
@@ -18,6 +18,8 @@
  */
 package hivemall.ensemble.bagging;
 
+import javax.annotation.Nullable;
+
 import org.apache.hadoop.hive.ql.exec.Description;
 import org.apache.hadoop.hive.ql.exec.UDAF;
 import org.apache.hadoop.hive.ql.exec.UDAFEvaluator;
@@ -54,11 +56,15 @@ public final class WeightVotedAvgUDAF extends UDAF {
             this.partial = null;
         }
 
-        public boolean iterate(double w) {
+        public boolean iterate(@Nullable DoubleWritable o) {
+            if (o == null) {
+                return true;
+            }
             if (partial == null) {
                 this.partial = new PartialResult();
                 partial.init();
             }
+            double w = o.get();
             if (w > 0) {
                 partial.positiveSum += w;
                 partial.positiveCnt++;

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/main/java/hivemall/ftvec/pairing/FeaturePairsUDTF.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/ftvec/pairing/FeaturePairsUDTF.java b/core/src/main/java/hivemall/ftvec/pairing/FeaturePairsUDTF.java
new file mode 100644
index 0000000..6aebd64
--- /dev/null
+++ b/core/src/main/java/hivemall/ftvec/pairing/FeaturePairsUDTF.java
@@ -0,0 +1,232 @@
+/*
+ * 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 hivemall.ftvec.pairing;
+
+import hivemall.UDTFWithOptions;
+import hivemall.model.FeatureValue;
+import hivemall.utils.hadoop.HiveUtils;
+import hivemall.utils.hashing.HashFunction;
+import hivemall.utils.lang.Preconditions;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import javax.annotation.Nonnull;
+
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Options;
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.serde2.io.DoubleWritable;
+import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.io.IntWritable;
+import org.apache.hadoop.io.Writable;
+
+@Description(name = "feature_pairs",
+        value = "_FUNC_(feature_vector in array<string>, [, const string options])"
+                + " - Returns a relation <string i, string j, double xi, double xj>")
+public final class FeaturePairsUDTF extends UDTFWithOptions {
+
+    private Type _type;
+    private RowProcessor _proc;
+
+    public FeaturePairsUDTF() {}
+
+    @Override
+    protected Options getOptions() {
+        Options opts = new Options();
+        opts.addOption("kpa", false,
+            "Generate feature pairs for Kernel-Expansion Passive Aggressive [default:true]");
+        opts.addOption("ffm", false,
+            "Generate feature pairs for Field-aware Factorization Machines [default:false]");
+        return opts;
+    }
+
+    @Override
+    protected CommandLine processOptions(ObjectInspector[] argOIs) throws UDFArgumentException {
+        CommandLine cl = null;
+        if (argOIs.length == 2) {
+            String args = HiveUtils.getConstString(argOIs[1]);
+            cl = parseOptions(args);
+
+            Preconditions.checkArgument(cl.getOptions().length == 1, UDFArgumentException.class,
+                "Only one option can be specified: " + cl.getArgList());
+
+            if (cl.hasOption("kpa")) {
+                this._type = Type.kpa;
+            } else if (cl.hasOption("ffm")) {
+                this._type = Type.ffm;
+            } else {
+                throw new UDFArgumentException("Unsupported option: " + cl.getArgList().get(0));
+            }
+        } else {
+            throw new UDFArgumentException("MUST provide -kpa or -ffm in the option");
+        }
+
+        return cl;
+    }
+
+    @Override
+    public StructObjectInspector initialize(ObjectInspector[] argOIs) throws UDFArgumentException {
+        if (argOIs.length != 1 && argOIs.length != 2) {
+            throw new UDFArgumentException("_FUNC_ takes 1 or 2 arguments");
+        }
+        processOptions(argOIs);
+
+        ListObjectInspector fvOI = HiveUtils.asListOI(argOIs[0]);
+        HiveUtils.validateFeatureOI(fvOI.getListElementObjectInspector());
+
+        final List<String> fieldNames = new ArrayList<String>(4);
+        final List<ObjectInspector> fieldOIs = new ArrayList<ObjectInspector>(4);
+        switch (_type) {
+            case kpa: {
+                this._proc = new KPAProcessor(fvOI);
+                fieldNames.add("h");
+                fieldOIs.add(PrimitiveObjectInspectorFactory.writableIntObjectInspector);
+                fieldNames.add("hk");
+                fieldOIs.add(PrimitiveObjectInspectorFactory.writableIntObjectInspector);
+                fieldNames.add("xh");
+                fieldOIs.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
+                fieldNames.add("xk");
+                fieldOIs.add(PrimitiveObjectInspectorFactory.writableDoubleObjectInspector);
+                break;
+            }
+            case ffm: {
+                throw new UDFArgumentException("-ffm is not supported yet");
+                //break;
+            }
+            default:
+                throw new UDFArgumentException("Illegal condition: " + _type);
+        }
+        return ObjectInspectorFactory.getStandardStructObjectInspector(fieldNames, fieldOIs);
+    }
+
+    @Override
+    public void process(Object[] args) throws HiveException {
+        Object arg0 = args[0];
+        if (arg0 == null) {
+            return;
+        }
+        _proc.process(arg0);
+    }
+
+    public enum Type {
+        kpa, ffm;
+    }
+
+    abstract class RowProcessor {
+
+        @Nonnull
+        protected final ListObjectInspector fvOI;
+
+        RowProcessor(@Nonnull ListObjectInspector fvOI) {
+            this.fvOI = fvOI;
+        }
+
+        void process(@Nonnull Object arg) throws HiveException {
+            final int size = fvOI.getListLength(arg);
+            if (size == 0) {
+                return;
+            }
+
+            final List<FeatureValue> features = new ArrayList<FeatureValue>(size);
+            for (int i = 0; i < size; i++) {
+                Object f = fvOI.getListElement(arg, i);
+                if (f == null) {
+                    continue;
+                }
+                FeatureValue fv = FeatureValue.parse(f, true);
+                features.add(fv);
+            }
+
+            process(features);
+        }
+
+        abstract void process(@Nonnull List<FeatureValue> features) throws HiveException;
+
+    }
+
+    final class KPAProcessor extends RowProcessor {
+
+        @Nonnull
+        private final IntWritable f0, f1;
+        @Nonnull
+        private final DoubleWritable f2, f3;
+        @Nonnull
+        private final Writable[] forward;
+
+        KPAProcessor(@Nonnull ListObjectInspector fvOI) {
+            super(fvOI);
+            this.f0 = new IntWritable();
+            this.f1 = new IntWritable();
+            this.f2 = new DoubleWritable();
+            this.f3 = new DoubleWritable();
+            this.forward = new Writable[] {f0, null, null, null};
+        }
+
+        @Override
+        void process(@Nonnull List<FeatureValue> features) throws HiveException {
+            forward[0] = f0;
+            f0.set(0);
+            forward[1] = null;
+            forward[2] = null;
+            forward[3] = null;
+            forward(forward); // forward h(f0)
+
+            forward[2] = f2;
+            for (int i = 0, len = features.size(); i < len; i++) {
+                FeatureValue xi = features.get(i);
+                int h = xi.getFeatureAsInt();
+                double xh = xi.getValue();
+                forward[0] = f0;
+                f0.set(h);
+                forward[1] = null;
+                f2.set(xh);
+                forward[3] = null;
+                forward(forward); // forward h(f0), xh(f2)
+
+                forward[0] = null;
+                forward[1] = f1;
+                forward[3] = f3;
+                for (int j = i + 1; j < len; j++) {
+                    FeatureValue xj = features.get(j);
+                    int k = xj.getFeatureAsInt();
+                    int hk = HashFunction.hash(h, k, true);
+                    double xk = xj.getValue();
+                    f1.set(hk);
+                    f3.set(xk);
+                    forward(forward);// forward hk(f1), xh(f2), xk(f3)
+                }
+            }
+        }
+    }
+
+
+    @Override
+    public void close() throws HiveException {
+        // clean up to help GC
+        this._proc = null;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/main/java/hivemall/model/FeatureValue.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/model/FeatureValue.java b/core/src/main/java/hivemall/model/FeatureValue.java
index 8f2b728..7ff3383 100644
--- a/core/src/main/java/hivemall/model/FeatureValue.java
+++ b/core/src/main/java/hivemall/model/FeatureValue.java
@@ -18,6 +18,9 @@
  */
 package hivemall.model;
 
+import hivemall.utils.hashing.MurmurHash3;
+import hivemall.utils.lang.Preconditions;
+
 import javax.annotation.Nonnull;
 import javax.annotation.Nullable;
 
@@ -44,6 +47,12 @@ public final class FeatureValue {
     public <T> T getFeature() {
         return (T) feature;
     }
+    
+    public int getFeatureAsInt() {
+        Preconditions.checkNotNull(feature);
+        Preconditions.checkArgument(feature instanceof Integer);
+        return ((Integer) feature).intValue();
+    }
 
     public double getValue() {
         return value;
@@ -63,30 +72,42 @@ public final class FeatureValue {
 
     @Nullable
     public static FeatureValue parse(final Object o) throws IllegalArgumentException {
+        return parse(o, false);
+    }
+
+    @Nullable
+    public static FeatureValue parse(final Object o, final boolean mhash)
+            throws IllegalArgumentException {
         if (o == null) {
             return null;
         }
         String s = o.toString();
-        return parse(s);
+        return parse(s, mhash);
     }
 
     @Nullable
     public static FeatureValue parse(@Nonnull final String s) throws IllegalArgumentException {
+        return parse(s, false);
+    }
+
+    @Nullable
+    public static FeatureValue parse(@Nonnull final String s, final boolean mhash)
+            throws IllegalArgumentException {
         assert (s != null);
         final int pos = s.indexOf(':');
         if (pos == 0) {
             throw new IllegalArgumentException("Invalid feature value representation: " + s);
         }
 
-        final Text feature;
+        final Object feature;
         final double weight;
         if (pos > 0) {
             String s1 = s.substring(0, pos);
             String s2 = s.substring(pos + 1);
-            feature = new Text(s1);
+            feature = mhash ? Integer.valueOf(MurmurHash3.murmurhash3(s1)) : new Text(s1);
             weight = Double.parseDouble(s2);
         } else {
-            feature = new Text(s);
+            feature = mhash ? Integer.valueOf(MurmurHash3.murmurhash3(s)) : new Text(s);
             weight = 1.d;
         }
         return new FeatureValue(feature, weight);

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/main/java/hivemall/utils/collections/FloatArrayList.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/collections/FloatArrayList.java b/core/src/main/java/hivemall/utils/collections/FloatArrayList.java
new file mode 100644
index 0000000..cfdf504
--- /dev/null
+++ b/core/src/main/java/hivemall/utils/collections/FloatArrayList.java
@@ -0,0 +1,152 @@
+/*
+ * 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 hivemall.utils.collections;
+
+import java.io.Serializable;
+
+public final class FloatArrayList implements Serializable {
+    private static final long serialVersionUID = 8764828070342317585L;
+
+    public static final int DEFAULT_CAPACITY = 12;
+
+    /** array entity */
+    private float[] data;
+    private int used;
+
+    public FloatArrayList() {
+        this(DEFAULT_CAPACITY);
+    }
+
+    public FloatArrayList(int size) {
+        this.data = new float[size];
+        this.used = 0;
+    }
+
+    public FloatArrayList(float[] initValues) {
+        this.data = initValues;
+        this.used = initValues.length;
+    }
+
+    public void add(float value) {
+        if (used >= data.length) {
+            expand(used + 1);
+        }
+        data[used++] = value;
+    }
+
+    public void add(float[] values) {
+        final int needs = used + values.length;
+        if (needs >= data.length) {
+            expand(needs);
+        }
+        System.arraycopy(values, 0, data, used, values.length);
+        this.used = needs;
+    }
+
+    /**
+     * dynamic expansion.
+     */
+    private void expand(int max) {
+        while (data.length < max) {
+            final int len = data.length;
+            float[] newArray = new float[len * 2];
+            System.arraycopy(data, 0, newArray, 0, len);
+            this.data = newArray;
+        }
+    }
+
+    public float remove() {
+        return data[--used];
+    }
+
+    public float remove(int index) {
+        final float ret;
+        if (index > used) {
+            throw new IndexOutOfBoundsException();
+        } else if (index == used) {
+            ret = data[--used];
+        } else { // index < used
+            // removed value
+            ret = data[index];
+            final float[] newarray = new float[--used];
+            // prefix
+            System.arraycopy(data, 0, newarray, 0, index - 1);
+            // appendix
+            System.arraycopy(data, index + 1, newarray, index, used - index);
+            // set fields.
+            this.data = newarray;
+        }
+        return ret;
+    }
+
+    public void set(int index, float value) {
+        if (index > used) {
+            throw new IllegalArgumentException("Index MUST be less than \"size()\".");
+        } else if (index == used) {
+            ++used;
+        }
+        data[index] = value;
+    }
+
+    public float get(int index) {
+        if (index >= used)
+            throw new IndexOutOfBoundsException();
+        return data[index];
+    }
+
+    public float fastGet(int index) {
+        return data[index];
+    }
+
+    public int size() {
+        return used;
+    }
+
+    public boolean isEmpty() {
+        return used == 0;
+    }
+
+    public void clear() {
+        this.used = 0;
+    }
+
+    public float[] toArray() {
+        final float[] newArray = new float[used];
+        System.arraycopy(data, 0, newArray, 0, used);
+        return newArray;
+    }
+
+    public float[] array() {
+        return data;
+    }
+
+    @Override
+    public String toString() {
+        final StringBuilder buf = new StringBuilder();
+        buf.append('[');
+        for (int i = 0; i < used; i++) {
+            if (i != 0) {
+                buf.append(", ");
+            }
+            buf.append(data[i]);
+        }
+        buf.append(']');
+        return buf.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/main/java/hivemall/utils/collections/Int2FloatOpenHashTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/collections/Int2FloatOpenHashTable.java b/core/src/main/java/hivemall/utils/collections/Int2FloatOpenHashTable.java
index 49f27c8..a06cdb0 100644
--- a/core/src/main/java/hivemall/utils/collections/Int2FloatOpenHashTable.java
+++ b/core/src/main/java/hivemall/utils/collections/Int2FloatOpenHashTable.java
@@ -73,6 +73,9 @@ public class Int2FloatOpenHashTable implements Externalizable {
         this(size, DEFAULT_LOAD_FACTOR, DEFAULT_GROW_FACTOR, true);
     }
 
+    /**
+     * Only for {@link Externalizable}
+     */
     public Int2FloatOpenHashTable() {// required for serialization
         this._loadFactor = DEFAULT_LOAD_FACTOR;
         this._growFactor = DEFAULT_GROW_FACTOR;

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/main/java/hivemall/utils/hadoop/HiveUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/hadoop/HiveUtils.java b/core/src/main/java/hivemall/utils/hadoop/HiveUtils.java
index 8188b7a..5423c9d 100644
--- a/core/src/main/java/hivemall/utils/hadoop/HiveUtils.java
+++ b/core/src/main/java/hivemall/utils/hadoop/HiveUtils.java
@@ -308,6 +308,18 @@ public final class HiveUtils {
         }
     }
 
+    public static boolean isStringTypeInfo(@Nonnull TypeInfo typeInfo) {
+        if (typeInfo.getCategory() != ObjectInspector.Category.PRIMITIVE) {
+            return false;
+        }
+        switch (((PrimitiveTypeInfo) typeInfo).getPrimitiveCategory()) {
+            case STRING:
+                return true;
+            default:
+                return false;
+        }
+    }
+
     public static boolean isConstString(@Nonnull final ObjectInspector oi) {
         return ObjectInspectorUtils.isConstantObjectInspector(oi) && isStringOI(oi);
     }
@@ -321,6 +333,20 @@ public final class HiveUtils {
         return (ListTypeInfo) typeInfo;
     }
 
+    public static float getFloat(@Nullable Object o, @Nonnull PrimitiveObjectInspector oi) {
+        if (o == null) {
+            return 0.f;
+        }
+        return PrimitiveObjectInspectorUtils.getFloat(o, oi);
+    }
+
+    public static double getDouble(@Nullable Object o, @Nonnull PrimitiveObjectInspector oi) {
+        if (o == null) {
+            return 0.d;
+        }
+        return PrimitiveObjectInspectorUtils.getDouble(o, oi);
+    }
+
     @SuppressWarnings("unchecked")
     @Nullable
     public static <T extends Writable> T getConstValue(@Nonnull final ObjectInspector oi)
@@ -776,6 +802,7 @@ public final class HiveUtils {
         return oi;
     }
 
+    @Nonnull
     public static PrimitiveObjectInspector asDoubleCompatibleOI(@Nonnull final ObjectInspector argOI)
             throws UDFArgumentTypeException {
         if (argOI.getCategory() != Category.PRIMITIVE) {
@@ -802,6 +829,30 @@ public final class HiveUtils {
     }
 
     @Nonnull
+    public static PrimitiveObjectInspector asNumberOI(@Nonnull final ObjectInspector argOI)
+            throws UDFArgumentTypeException {
+        if (argOI.getCategory() != Category.PRIMITIVE) {
+            throw new UDFArgumentTypeException(0, "Only primitive type arguments are accepted but "
+                    + argOI.getTypeName() + " is passed.");
+        }
+        final PrimitiveObjectInspector oi = (PrimitiveObjectInspector) argOI;
+        switch (oi.getPrimitiveCategory()) {
+            case BYTE:
+            case SHORT:
+            case INT:
+            case LONG:
+            case FLOAT:
+            case DOUBLE:
+                break;
+            default:
+                throw new UDFArgumentTypeException(0,
+                    "Only numeric or string type arguments are accepted but " + argOI.getTypeName()
+                            + " is passed.");
+        }
+        return oi;
+    }
+
+    @Nonnull
     public static ListObjectInspector asListOI(@Nonnull final ObjectInspector oi)
             throws UDFArgumentException {
         Category category = oi.getCategory();

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/main/java/hivemall/utils/hashing/HashFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/hashing/HashFunction.java b/core/src/main/java/hivemall/utils/hashing/HashFunction.java
index eb44915..bd1b841 100644
--- a/core/src/main/java/hivemall/utils/hashing/HashFunction.java
+++ b/core/src/main/java/hivemall/utils/hashing/HashFunction.java
@@ -18,6 +18,8 @@
  */
 package hivemall.utils.hashing;
 
+import hivemall.utils.math.MathUtils;
+
 public abstract class HashFunction {
 
     public int hash(Object data) {
@@ -25,6 +27,18 @@ public abstract class HashFunction {
         return hash(s);
     }
 
+    public static int hash(final int first, final int second, final boolean positive) {
+        final int h = first * 157 + second;
+        if (positive) {
+            int r = MathUtils.moduloPowerOfTwo(h, MurmurHash3.DEFAULT_NUM_FEATURES);
+            if (r < 0) {
+                r += MurmurHash3.DEFAULT_NUM_FEATURES;
+            }
+            return r;
+        }
+        return h;
+    }
+
     public abstract int hash(String data);
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/main/java/hivemall/utils/lang/Preconditions.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/lang/Preconditions.java b/core/src/main/java/hivemall/utils/lang/Preconditions.java
index af63127..eabbc0a 100644
--- a/core/src/main/java/hivemall/utils/lang/Preconditions.java
+++ b/core/src/main/java/hivemall/utils/lang/Preconditions.java
@@ -98,19 +98,21 @@ public final class Preconditions {
     }
 
     public static <E extends Throwable> void checkArgument(boolean expression,
-            @Nonnull String errorMessage, @Nonnull Class<E> clazz) throws E {
+            @Nonnull Class<E> clazz, @Nullable Object errorMessage) throws E {
         if (!expression) {
+            Constructor<E> constructor;
+            try {
+                constructor = clazz.getConstructor(String.class);
+            } catch (NoSuchMethodException | SecurityException e) {
+                throw new IllegalStateException(
+                    "Failed to get a constructor of " + clazz.getName(), e);
+            }
             final E throwable;
             try {
-                Constructor<E> constructor = clazz.getConstructor(String.class);
                 throwable = constructor.newInstance(errorMessage);
-            } catch (NoSuchMethodException | SecurityException e1) {
-                throw new IllegalStateException("Failed to get a Constructor(String): "
-                        + clazz.getName(), e1);
-            } catch (InstantiationException | IllegalAccessException | IllegalArgumentException
-                    | InvocationTargetException e2) {
+            } catch (ReflectiveOperationException | IllegalArgumentException e) {
                 throw new IllegalStateException(
-                    "Failed to instantiate a class: " + clazz.getName(), e2);
+                    "Failed to instantiate a class: " + clazz.getName(), e);
             }
             throw throwable;
         }

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/main/java/hivemall/utils/math/MathUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/hivemall/utils/math/MathUtils.java b/core/src/main/java/hivemall/utils/math/MathUtils.java
index ae9f029..252ccf6 100644
--- a/core/src/main/java/hivemall/utils/math/MathUtils.java
+++ b/core/src/main/java/hivemall/utils/math/MathUtils.java
@@ -89,8 +89,7 @@ public final class MathUtils {
     }
 
     /**
-     * <a href="https://en.wikipedia.org/wiki/Logit">Logit</a> is the inverse of
-     * {@link #sigmoid(double)} function.
+     * <a href="https://en.wikipedia.org/wiki/Logit">Logit</a> is the inverse of {@link #sigmoid(double)} function.
      */
     public static double logit(final double p) {
         return Math.log(p / (1.d - p));
@@ -101,14 +100,11 @@ public final class MathUtils {
     }
 
     /**
-     * Returns the inverse erf. This code is based on erfInv() in
-     * org.apache.commons.math3.special.Erf.
+     * Returns the inverse erf. This code is based on erfInv() in org.apache.commons.math3.special.Erf.
      * <p>
-     * This implementation is described in the paper: <a
-     * href="http://people.maths.ox.ac.uk/gilesm/files/gems_erfinv.pdf">Approximating the erfinv
-     * function</a> by Mike Giles, Oxford-Man Institute of Quantitative Finance, which was published
-     * in GPU Computing Gems, volume 2, 2010. The source code is available <a
-     * href="http://gpucomputing.net/?q=node/1828">here</a>.
+     * This implementation is described in the paper: <a href="http://people.maths.ox.ac.uk/gilesm/files/gems_erfinv.pdf">Approximating the erfinv
+     * function</a> by Mike Giles, Oxford-Man Institute of Quantitative Finance, which was published in GPU Computing Gems, volume 2, 2010. The source
+     * code is available <a href="http://gpucomputing.net/?q=node/1828">here</a>.
      * </p>
      * 
      * @param x the value
@@ -227,8 +223,8 @@ public final class MathUtils {
         return v < 0 ? -1 : 1;
     }
 
-    public static float sign(final float v) {
-        return v < 0.f ? -1.f : 1.f;
+    public static int sign(final float v) {
+        return v < 0.f ? -1 : 1;
     }
 
     public static double log(final double n, final int base) {

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/test/java/hivemall/anomaly/ChangeFinder1DTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/anomaly/ChangeFinder1DTest.java b/core/src/test/java/hivemall/anomaly/ChangeFinder1DTest.java
index d7bac75..1470ffb 100644
--- a/core/src/test/java/hivemall/anomaly/ChangeFinder1DTest.java
+++ b/core/src/test/java/hivemall/anomaly/ChangeFinder1DTest.java
@@ -46,7 +46,7 @@ public class ChangeFinder1DTest {
         ChangeFinder1D cf = new ChangeFinder1D(params, oi);
         double[] outScores = new double[2];
 
-        BufferedReader reader = readFile("cf1d.csv");
+        BufferedReader reader = readFile("cf1d.csv.gz");
         println("x outlier change");
         String line;
         int numOutliers = 0, numChangepoints = 0;

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/test/java/hivemall/anomaly/ChangeFinder2DTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/anomaly/ChangeFinder2DTest.java b/core/src/test/java/hivemall/anomaly/ChangeFinder2DTest.java
index 240906c..43a0921 100644
--- a/core/src/test/java/hivemall/anomaly/ChangeFinder2DTest.java
+++ b/core/src/test/java/hivemall/anomaly/ChangeFinder2DTest.java
@@ -60,7 +60,7 @@ public class ChangeFinder2DTest {
         double[] outScores = new double[2];
         List<Double> x = new ArrayList<Double>(1);
 
-        BufferedReader reader = readFile("cf1d.csv");
+        BufferedReader reader = readFile("cf1d.csv.gz");
         println("x outlier change");
         String line;
         int i = 1, numOutliers = 0, numChangepoints = 0;

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/test/java/hivemall/classifier/KernelExpansionPassiveAggressiveUDTFTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/classifier/KernelExpansionPassiveAggressiveUDTFTest.java b/core/src/test/java/hivemall/classifier/KernelExpansionPassiveAggressiveUDTFTest.java
new file mode 100644
index 0000000..a5c98e1
--- /dev/null
+++ b/core/src/test/java/hivemall/classifier/KernelExpansionPassiveAggressiveUDTFTest.java
@@ -0,0 +1,158 @@
+/*
+ * 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 hivemall.classifier;
+
+import hivemall.model.FeatureValue;
+import hivemall.utils.math.MathUtils;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.StringTokenizer;
+import java.util.zip.GZIPInputStream;
+
+import javax.annotation.Nonnull;
+
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.serde2.objectinspector.ListObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspectorUtils;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class KernelExpansionPassiveAggressiveUDTFTest {
+
+    @Test
+    public void testNews20() throws IOException, ParseException, HiveException {
+        KernelExpansionPassiveAggressiveUDTF udtf = new KernelExpansionPassiveAggressiveUDTF();
+        ObjectInspector intOI = PrimitiveObjectInspectorFactory.javaIntObjectInspector;
+        ObjectInspector stringOI = PrimitiveObjectInspectorFactory.javaStringObjectInspector;
+        ListObjectInspector stringListOI = ObjectInspectorFactory.getStandardListObjectInspector(stringOI);
+        udtf.initialize(new ObjectInspector[] {stringListOI, intOI});
+
+        BufferedReader news20 = readFile("news20-small.binary.gz");
+        ArrayList<String> words = new ArrayList<String>();
+        String line = news20.readLine();
+        while (line != null) {
+            StringTokenizer tokens = new StringTokenizer(line, " ");
+            int label = Integer.parseInt(tokens.nextToken());
+            while (tokens.hasMoreTokens()) {
+                words.add(tokens.nextToken());
+            }
+            Assert.assertFalse(words.isEmpty());
+            udtf.process(new Object[] {words, label});
+
+            words.clear();
+            line = news20.readLine();
+        }
+
+        Assert.assertTrue(Math.abs(udtf.getLoss()) < 0.1f);
+
+        news20.close();
+    }
+
+    public void test_a9a() throws IOException, ParseException, HiveException {
+        KernelExpansionPassiveAggressiveUDTF udtf = new KernelExpansionPassiveAggressiveUDTF();
+        ObjectInspector intOI = PrimitiveObjectInspectorFactory.javaIntObjectInspector;
+        ObjectInspector stringOI = PrimitiveObjectInspectorFactory.javaStringObjectInspector;
+        ListObjectInspector stringListOI = ObjectInspectorFactory.getStandardListObjectInspector(stringOI);
+        ObjectInspector params = ObjectInspectorUtils.getConstantObjectInspector(
+            PrimitiveObjectInspectorFactory.javaStringObjectInspector, "-c 0.01");
+
+        udtf.initialize(new ObjectInspector[] {stringListOI, intOI, params});
+
+        final ArrayList<String> words = new ArrayList<String>();
+        BufferedReader trainData = readFile("a9a.gz");
+        String line = trainData.readLine();
+        while (line != null) {
+            StringTokenizer tokens = new StringTokenizer(line, " ");
+            String labelStr = tokens.nextToken();
+            final int label;
+            if ("+1".equals(labelStr)) {
+                label = 1;
+            } else if ("-1".equals(labelStr)) {
+                label = -1;
+            } else {
+                throw new IllegalStateException("Illegal label: " + labelStr);
+            }
+            while (tokens.hasMoreTokens()) {
+                words.add(tokens.nextToken());
+            }
+            Assert.assertFalse(words.isEmpty());
+            udtf.process(new Object[] {words, label});
+
+            words.clear();
+            line = trainData.readLine();
+        }
+        trainData.close();
+
+        int numTests = 0;
+        int numCorrect = 0;
+
+        BufferedReader testData = readFile("a9a.t.gz");
+        line = testData.readLine();
+        while (line != null) {
+            StringTokenizer tokens = new StringTokenizer(line, " ");
+            String labelStr = tokens.nextToken();
+            final int actual;
+            if ("+1".equals(labelStr)) {
+                actual = 1;
+            } else if ("-1".equals(labelStr)) {
+                actual = -1;
+            } else {
+                throw new IllegalStateException("Illegal label: " + labelStr);
+            }
+            while (tokens.hasMoreTokens()) {
+                words.add(tokens.nextToken());
+            }
+            Assert.assertFalse(words.isEmpty());
+
+            FeatureValue[] features = udtf.parseFeatures(words);
+            float score = udtf.predict(features);
+            int predicted = MathUtils.sign(score);
+
+            if (predicted == actual) {
+                ++numCorrect;
+            }
+            ++numTests;
+
+            words.clear();
+            line = testData.readLine();
+        }
+        testData.close();
+
+        float accuracy = numCorrect / (float) numTests;
+        Assert.assertTrue(accuracy > 0.82f);
+    }
+
+    @Nonnull
+    private static BufferedReader readFile(@Nonnull String fileName) throws IOException {
+        InputStream is = KernelExpansionPassiveAggressiveUDTFTest.class.getResourceAsStream(fileName);
+        if (fileName.endsWith(".gz")) {
+            is = new GZIPInputStream(is);
+        }
+        return new BufferedReader(new InputStreamReader(is));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/test/java/hivemall/fm/FactorizationMachineUDTFTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/fm/FactorizationMachineUDTFTest.java b/core/src/test/java/hivemall/fm/FactorizationMachineUDTFTest.java
index eacfa8d..6d053de 100644
--- a/core/src/test/java/hivemall/fm/FactorizationMachineUDTFTest.java
+++ b/core/src/test/java/hivemall/fm/FactorizationMachineUDTFTest.java
@@ -20,10 +20,14 @@ package hivemall.fm;
 
 import java.io.BufferedReader;
 import java.io.IOException;
+import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.StringTokenizer;
+import java.util.zip.GZIPInputStream;
+
+import javax.annotation.Nonnull;
 
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
@@ -56,8 +60,7 @@ public class FactorizationMachineUDTFTest {
         double loss = 0.d;
         double cumul = 0.d;
         for (int trainingIteration = 1; trainingIteration <= ITERATIONS; ++trainingIteration) {
-            BufferedReader data = new BufferedReader(new InputStreamReader(
-                getClass().getResourceAsStream("5107786.txt")));
+            BufferedReader data = readFile("5107786.txt.gz");
             loss = udtf._cvState.getCumulativeLoss();
             int trExamples = 0;
             String line = data.readLine();
@@ -78,8 +81,17 @@ public class FactorizationMachineUDTFTest {
             println(trainingIteration + " " + loss + " " + cumul / (trainingIteration * trExamples));
             data.close();
         }
+
         Assert.assertTrue("Loss was greater than 0.1: " + loss, loss <= 0.1);
+    }
 
+    @Nonnull
+    private static BufferedReader readFile(@Nonnull String fileName) throws IOException {
+        InputStream is = FactorizationMachineUDTFTest.class.getResourceAsStream(fileName);
+        if (fileName.endsWith(".gz")) {
+            is = new GZIPInputStream(is);
+        }
+        return new BufferedReader(new InputStreamReader(is));
     }
 
     private static void println(String line) {

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/test/java/hivemall/fm/FieldAwareFactorizationMachineUDTFTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/fm/FieldAwareFactorizationMachineUDTFTest.java b/core/src/test/java/hivemall/fm/FieldAwareFactorizationMachineUDTFTest.java
index f4d4f80..792ede1 100644
--- a/core/src/test/java/hivemall/fm/FieldAwareFactorizationMachineUDTFTest.java
+++ b/core/src/test/java/hivemall/fm/FieldAwareFactorizationMachineUDTFTest.java
@@ -20,8 +20,12 @@ package hivemall.fm;
 
 import java.io.BufferedReader;
 import java.io.IOException;
+import java.io.InputStream;
 import java.io.InputStreamReader;
 import java.util.ArrayList;
+import java.util.zip.GZIPInputStream;
+
+import javax.annotation.Nonnull;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -87,8 +91,7 @@ public class FieldAwareFactorizationMachineUDTFTest {
         double loss = 0.d;
         double cumul = 0.d;
         for (int trainingIteration = 1; trainingIteration <= ITERATIONS; ++trainingIteration) {
-            BufferedReader data = new BufferedReader(new InputStreamReader(
-                FieldAwareFactorizationMachineUDTFTest.class.getResourceAsStream("bigdata.tr.txt")));
+            BufferedReader data = readFile("bigdata.tr.txt.gz");
             loss = udtf._cvState.getCumulativeLoss();
             int lines = 0;
             for (int lineNumber = 0; lineNumber < MAX_LINES; ++lineNumber, ++lines) {
@@ -131,6 +134,15 @@ public class FieldAwareFactorizationMachineUDTFTest {
         Assert.assertTrue("Last loss was greater than expected: " + loss, loss < lossThreshold);
     }
 
+    @Nonnull
+    private static BufferedReader readFile(@Nonnull String fileName) throws IOException {
+        InputStream is = FieldAwareFactorizationMachineUDTFTest.class.getResourceAsStream(fileName);
+        if (fileName.endsWith(".gz")) {
+            is = new GZIPInputStream(is);
+        }
+        return new BufferedReader(new InputStreamReader(is));
+    }
+
     private static String[] toStringArray(ArrayList<StringFeature> x) {
         final int size = x.size();
         final String[] ret = new String[size];

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/test/java/hivemall/mf/BPRMatrixFactorizationUDTFTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/mf/BPRMatrixFactorizationUDTFTest.java b/core/src/test/java/hivemall/mf/BPRMatrixFactorizationUDTFTest.java
index e28d318..41f1f97 100644
--- a/core/src/test/java/hivemall/mf/BPRMatrixFactorizationUDTFTest.java
+++ b/core/src/test/java/hivemall/mf/BPRMatrixFactorizationUDTFTest.java
@@ -18,11 +18,13 @@
  */
 package hivemall.mf;
 
-import hivemall.utils.io.IOUtils;
 import hivemall.utils.lang.StringUtils;
 
 import java.io.BufferedReader;
 import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.util.zip.GZIPInputStream;
 
 import javax.annotation.Nonnull;
 
@@ -63,7 +65,7 @@ public class BPRMatrixFactorizationUDTFTest {
         final IntWritable negItem = new IntWritable();
         final Object[] args = new Object[] {user, posItem, negItem};
 
-        BufferedReader train = IOUtils.bufferedReader(getClass().getResourceAsStream("ml1k.train"));
+        BufferedReader train = readFile("ml1k.train.gz");
         String line;
         while ((line = train.readLine()) != null) {
             parseLine(line, user, posItem, negItem);
@@ -98,7 +100,7 @@ public class BPRMatrixFactorizationUDTFTest {
         final IntWritable negItem = new IntWritable();
         final Object[] args = new Object[] {user, posItem, negItem};
 
-        BufferedReader train = IOUtils.bufferedReader(getClass().getResourceAsStream("ml1k.train"));
+        BufferedReader train = readFile("ml1k.train.gz");
         String line;
         while ((line = train.readLine()) != null) {
             parseLine(line, user, posItem, negItem);
@@ -109,6 +111,14 @@ public class BPRMatrixFactorizationUDTFTest {
         Assert.assertTrue("finishedIter: " + finishedIter, finishedIter < iterations);
     }
 
+    @Nonnull
+    private static BufferedReader readFile(@Nonnull String fileName) throws IOException {
+        InputStream is = BPRMatrixFactorizationUDTFTest.class.getResourceAsStream(fileName);
+        if (fileName.endsWith(".gz")) {
+            is = new GZIPInputStream(is);
+        }
+        return new BufferedReader(new InputStreamReader(is));
+    }
 
     private static void parseLine(@Nonnull String line, @Nonnull IntWritable user,
             @Nonnull IntWritable posItem, @Nonnull IntWritable negItem) {

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/test/java/hivemall/utils/collections/Int2FloatOpenHashMapTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/utils/collections/Int2FloatOpenHashMapTest.java b/core/src/test/java/hivemall/utils/collections/Int2FloatOpenHashMapTest.java
index 14635be..8a8a68d 100644
--- a/core/src/test/java/hivemall/utils/collections/Int2FloatOpenHashMapTest.java
+++ b/core/src/test/java/hivemall/utils/collections/Int2FloatOpenHashMapTest.java
@@ -80,4 +80,17 @@ public class Int2FloatOpenHashMapTest {
         Assert.assertEquals(-1, itor.next());
     }
 
+    @Test
+    public void testIterator2() {
+        Int2FloatOpenHashTable map = new Int2FloatOpenHashTable(100);
+        map.put(33, 3.16f);
+
+        Int2FloatOpenHashTable.IMapIterator itor = map.entries();
+        Assert.assertTrue(itor.hasNext());
+        Assert.assertNotEquals(-1, itor.next());
+        Assert.assertEquals(33, itor.getKey());
+        Assert.assertEquals(3.16f, itor.getValue(), 0.d);
+        Assert.assertEquals(-1, itor.next());
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/test/java/hivemall/utils/io/Base91OutputStreamTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/utils/io/Base91OutputStreamTest.java b/core/src/test/java/hivemall/utils/io/Base91OutputStreamTest.java
index 99a132f..c7f41f9 100644
--- a/core/src/test/java/hivemall/utils/io/Base91OutputStreamTest.java
+++ b/core/src/test/java/hivemall/utils/io/Base91OutputStreamTest.java
@@ -50,7 +50,7 @@ public class Base91OutputStreamTest {
 
     @Test
     public void testLargeEncodeOutDecodeIn() throws IOException {
-        InputStream in = ArrayModelTest.class.getResourceAsStream("bigdata.tr.txt");
+        InputStream in = ArrayModelTest.class.getResourceAsStream("bigdata.tr.txt.gz");
         byte[] expected = IOUtils.toByteArray(in);
 
         FastByteArrayOutputStream bos = new FastByteArrayOutputStream();

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/test/java/hivemall/utils/lang/PreconditionsTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/hivemall/utils/lang/PreconditionsTest.java b/core/src/test/java/hivemall/utils/lang/PreconditionsTest.java
index 4cbbd3f..cf27a75 100644
--- a/core/src/test/java/hivemall/utils/lang/PreconditionsTest.java
+++ b/core/src/test/java/hivemall/utils/lang/PreconditionsTest.java
@@ -52,7 +52,7 @@ public class PreconditionsTest {
     public void testCheckArgumentBooleanClassOfE2() {
         final String msg = "safdfvzfd";
         try {
-            Preconditions.checkArgument(false, msg, HiveException.class);
+            Preconditions.checkArgument(false, HiveException.class, msg);
         } catch (HiveException e) {
             if (e.getMessage().equals(msg)) {
                 return;


[3/6] incubator-hivemall git commit: Close #13: Implement Kernel Expansion Passive Aggressive Classification

Posted by my...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/test/resources/hivemall/mf/ml1k.test
----------------------------------------------------------------------
diff --git a/core/src/test/resources/hivemall/mf/ml1k.test b/core/src/test/resources/hivemall/mf/ml1k.test
deleted file mode 100644
index e00b112..0000000
--- a/core/src/test/resources/hivemall/mf/ml1k.test
+++ /dev/null
@@ -1,20000 +0,0 @@
-676 345 2
-804 526 4
-476 168 5
-194 99 3
-846 474 5
-782 269 3
-932 1050 4
-935 476 4
-712 418 3
-3 339 3
-13 684 5
-222 763 3
-851 174 5
-472 318 5
-535 495 3
-83 1028 4
-707 792 4
-42 168 3
-26 1015 3
-60 601 4
-881 208 3
-682 117 4
-906 221 4
-705 416 3
-398 71 5
-58 475 5
-868 155 2
-405 440 1
-253 328 4
-880 976 2
-574 1313 4
-312 228 3
-11 111 4
-442 318 4
-521 829 2
-58 850 5
-514 582 4
-234 1169 4
-759 332 4
-56 575 3
-561 71 2
-943 92 5
-301 133 4
-82 519 4
-773 948 2
-136 257 3
-345 696 3
-194 70 3
-324 749 3
-435 214 4
-421 82 4
-591 451 3
-13 612 4
-653 670 1
-181 412 2
-327 133 4
-614 235 5
-942 607 5
-577 566 4
-833 328 2
-601 427 4
-488 230 3
-324 301 5
-16 66 4
-72 134 5
-346 571 3
-527 127 5
-235 431 2
-286 11 5
-211 257 5
-305 98 4
-892 1124 4
-694 205 5
-896 257 4
-321 45 4
-835 196 5
-214 302 4
-276 420 4
-185 237 4
-373 162 3
-416 238 4
-394 232 4
-82 235 1
-218 591 3
-588 234 5
-305 200 3
-929 187 5
-881 417 2
-107 271 2
-90 1205 3
-201 318 5
-289 477 2
-60 61 4
-653 546 2
-251 596 3
-232 50 4
-234 1330 3
-592 202 5
-710 187 5
-318 14 4
-314 1041 4
-169 174 4
-498 293 4
-106 9 4
-405 1437 1
-308 715 5
-592 58 5
-342 1011 3
-796 371 5
-3 318 4
-766 518 3
-916 382 4
-99 410 5
-675 347 4
-262 238 4
-749 468 3
-529 876 3
-883 289 5
-90 498 5
-862 505 4
-760 748 4
-526 248 4
-151 131 5
-760 819 1
-399 273 3
-334 430 4
-49 238 4
-148 204 3
-130 555 4
-214 137 4
-697 324 5
-580 323 2
-472 1228 4
-307 210 2
-140 880 4
-94 631 5
-738 109 4
-679 172 5
-178 607 3
-318 257 5
-506 216 4
-2 281 3
-717 1051 3
-115 284 2
-75 56 5
-429 367 3
-305 1286 5
-748 300 4
-276 475 5
-115 124 5
-225 64 4
-449 515 5
-389 132 5
-847 455 2
-877 197 4
-906 321 4
-554 98 5
-489 312 2
-511 887 5
-733 924 4
-632 144 4
-298 99 3
-232 175 5
-934 755 4
-234 610 4
-60 47 4
-22 1001 1
-875 171 5
-457 70 4
-465 181 3
-655 966 3
-650 631 3
-303 679 2
-798 231 2
-189 135 4
-174 126 5
-276 187 5
-250 184 1
-59 240 2
-246 82 2
-85 212 2
-856 323 2
-307 168 5
-450 847 4
-823 48 5
-361 655 3
-318 531 4
-429 419 4
-94 746 4
-283 109 4
-262 65 4
-62 1136 3
-507 894 5
-109 655 3
-727 111 3
-881 226 3
-708 1152 5
-286 640 5
-537 514 4
-923 117 4
-403 928 3
-748 847 4
-927 928 4
-295 47 5
-925 245 3
-130 427 5
-460 221 4
-447 248 5
-927 222 5
-606 50 5
-130 28 4
-645 39 3
-23 196 2
-518 1114 2
-828 20 2
-313 326 4
-455 304 3
-843 211 2
-347 1039 5
-401 121 3
-883 955 5
-332 294 5
-673 326 4
-701 286 4
-933 317 4
-892 168 4
-846 705 3
-452 194 4
-423 125 2
-798 732 2
-349 25 3
-363 95 3
-121 165 4
-782 266 1
-216 55 5
-474 607 4
-421 11 2
-6 306 4
-151 629 4
-694 684 4
-586 215 4
-328 29 3
-461 285 4
-328 510 5
-95 179 3
-13 263 5
-892 1454 3
-913 423 3
-673 313 4
-194 629 3
-574 346 4
-605 191 5
-236 69 5
-334 272 4
-702 258 5
-130 215 5
-23 185 4
-327 204 4
-524 47 2
-269 5 2
-450 549 3
-130 236 5
-297 20 4
-140 873 2
-9 340 4
-586 228 3
-76 343 3
-269 50 3
-587 689 1
-553 482 4
-646 286 3
-669 357 4
-766 494 3
-796 447 3
-180 660 5
-268 403 4
-482 258 2
-82 71 4
-711 354 3
-627 699 1
-239 64 1
-540 258 4
-374 1001 1
-926 313 3
-389 608 3
-819 268 4
-40 345 4
-456 721 4
-379 157 4
-100 288 2
-317 326 3
-676 344 5
-234 964 4
-559 56 3
-923 338 4
-466 144 5
-530 607 5
-456 747 4
-500 319 4
-429 121 3
-363 163 3
-747 305 5
-286 329 4
-699 202 3
-44 153 4
-239 511 5
-429 300 3
-896 1248 2
-448 1602 4
-682 76 3
-582 50 5
-936 221 4
-454 531 2
-330 44 5
-503 277 4
-514 651 4
-504 418 3
-642 624 3
-707 949 3
-653 77 3
-751 481 4
-453 797 1
-456 720 3
-611 347 4
-624 358 3
-839 276 3
-672 50 3
-59 235 1
-901 78 4
-29 689 2
-790 83 3
-684 596 3
-642 90 4
-640 318 5
-868 202 3
-624 979 4
-652 300 4
-69 334 3
-85 1166 4
-68 409 3
-232 186 4
-836 268 3
-890 194 5
-303 479 5
-70 206 3
-64 1133 4
-805 831 4
-724 893 3
-733 1142 4
-94 471 4
-184 1398 5
-803 269 5
-874 137 4
-3 342 4
-608 886 1
-7 648 5
-468 531 4
-683 303 3
-856 748 3
-437 1090 1
-145 15 2
-897 193 3
-611 882 4
-514 402 4
-269 174 1
-334 28 3
-492 172 3
-805 5 4
-463 1067 2
-72 82 3
-409 1050 4
-57 1059 3
-606 924 5
-450 481 5
-929 435 3
-308 69 2
-280 1049 2
-806 157 3
-530 204 4
-797 269 3
-929 197 3
-757 230 4
-698 435 3
-817 363 3
-492 291 4
-776 590 1
-393 1178 3
-734 230 2
-457 228 5
-696 310 4
-56 174 5
-394 79 5
-643 1149 3
-689 358 4
-15 255 5
-269 111 1
-862 22 5
-368 7 4
-83 720 4
-889 550 3
-500 58 3
-894 355 3
-495 403 5
-254 951 4
-277 137 3
-307 509 3
-383 425 4
-915 315 4
-771 4 1
-778 150 3
-446 690 2
-293 22 3
-934 786 1
-378 226 3
-210 274 5
-533 20 5
-59 448 4
-43 1055 2
-747 99 5
-849 928 5
-13 912 2
-707 1163 4
-758 313 4
-119 316 4
-561 544 2
-151 480 5
-532 404 5
-552 1051 3
-881 281 3
-327 228 4
-551 209 5
-174 458 4
-201 566 3
-846 495 4
-843 434 4
-118 641 5
-116 350 3
-430 264 2
-445 204 3
-862 205 4
-884 127 4
-269 447 3
-198 824 2
-378 742 4
-49 10 3
-207 191 4
-455 694 4
-796 496 5
-271 215 4
-860 514 5
-747 28 4
-445 28 4
-409 98 5
-749 50 5
-286 16 3
-803 286 5
-11 414 3
-608 319 4
-395 257 5
-476 433 4
-582 151 4
-429 559 3
-757 254 2
-213 151 5
-676 114 5
-592 1071 4
-699 20 4
-222 366 4
-658 198 5
-622 722 3
-727 366 3
-24 402 4
-870 69 4
-499 1101 5
-207 293 2
-82 479 4
-650 2 3
-631 332 3
-921 228 3
-1 250 4
-899 174 5
-149 245 3
-903 1101 4
-299 111 3
-11 714 4
-654 218 2
-119 154 5
-535 338 3
-603 1483 5
-435 121 3
-606 196 4
-264 150 5
-472 71 2
-464 294 4
-176 246 5
-328 645 4
-756 245 3
-918 382 4
-655 1644 1
-234 657 4
-395 315 5
-238 286 5
-95 290 3
-75 291 1
-865 24 4
-178 483 4
-479 213 4
-924 836 3
-887 1 5
-870 1 5
-854 461 3
-455 71 3
-336 288 3
-181 303 1
-239 209 5
-850 121 5
-606 428 3
-648 28 5
-59 230 4
-868 1037 1
-267 293 4
-695 343 4
-311 581 3
-458 193 4
-305 628 4
-546 672 3
-487 402 4
-566 219 1
-308 568 5
-478 134 2
-130 566 4
-592 460 3
-87 2 4
-671 176 2
-664 153 4
-312 919 3
-181 833 1
-244 70 4
-660 125 3
-854 238 5
-181 277 1
-81 405 3
-334 222 4
-548 270 5
-835 174 5
-158 241 4
-239 89 4
-528 588 2
-222 1079 1
-773 895 2
-896 730 4
-194 467 5
-894 170 4
-907 696 5
-260 1105 5
-684 118 4
-901 728 4
-614 100 5
-327 508 2
-715 196 4
-240 269 5
-82 1162 1
-829 515 4
-317 322 3
-426 489 5
-168 405 4
-788 229 3
-470 124 3
-479 523 4
-13 384 2
-838 311 4
-344 272 5
-878 194 4
-932 429 5
-201 146 1
-652 288 2
-13 770 4
-151 204 4
-21 993 4
-435 227 4
-56 226 4
-382 150 2
-313 191 5
-97 189 4
-715 234 4
-745 480 3
-276 214 5
-486 1143 3
-354 238 4
-682 268 5
-899 291 4
-379 522 5
-262 293 2
-846 110 3
-648 174 5
-501 840 4
-696 899 3
-135 56 4
-374 930 2
-856 749 3
-49 7 4
-303 160 4
-297 272 5
-599 1278 5
-344 297 4
-108 181 3
-524 1101 4
-339 1404 5
-890 142 3
-397 513 5
-174 82 1
-269 483 4
-234 274 3
-758 82 4
-826 684 3
-294 313 5
-313 175 4
-290 1285 3
-805 105 2
-894 919 4
-806 155 3
-524 367 5
-848 479 5
-632 172 5
-280 483 4
-312 175 3
-932 479 5
-807 195 3
-256 1057 2
-551 552 3
-12 238 5
-352 657 4
-270 218 5
-160 463 4
-374 986 3
-56 29 3
-275 825 2
-313 69 5
-435 462 5
-406 654 4
-249 183 4
-487 288 4
-294 831 3
-757 559 4
-766 196 3
-394 568 5
-606 684 3
-899 188 2
-262 125 3
-303 17 4
-321 198 4
-495 153 5
-916 172 5
-608 218 4
-399 31 3
-705 720 5
-181 1353 1
-4 354 5
-816 1025 4
-190 696 3
-151 584 3
-387 217 3
-699 929 3
-489 269 3
-391 294 2
-346 944 3
-52 280 3
-524 6 5
-678 282 3
-74 124 3
-206 1431 1
-622 501 3
-314 932 4
-178 260 1
-217 181 1
-862 974 2
-8 233 4
-749 523 4
-193 328 3
-474 605 3
-405 37 1
-49 217 3
-648 176 4
-255 452 3
-643 603 5
-109 54 3
-159 412 3
-543 64 4
-373 22 5
-435 1552 3
-671 583 3
-505 31 4
-7 171 3
-57 988 4
-870 382 3
-869 315 3
-145 218 3
-184 7 3
-301 294 4
-627 740 1
-298 603 5
-787 271 1
-627 658 3
-184 1020 4
-334 607 3
-186 1385 2
-542 121 2
-373 102 5
-113 7 3
-249 218 3
-880 12 5
-463 20 5
-218 186 3
-336 692 3
-136 744 5
-60 633 4
-59 193 4
-92 500 4
-472 4 3
-402 126 4
-354 882 4
-773 238 4
-758 109 3
-136 515 5
-722 333 5
-747 274 4
-694 632 4
-223 25 1
-410 352 3
-453 732 3
-815 837 5
-426 418 3
-930 14 4
-766 602 4
-49 1072 1
-894 628 3
-11 175 3
-823 202 4
-336 796 3
-313 144 4
-244 660 4
-542 100 4
-389 477 4
-661 471 4
-718 289 3
-202 96 4
-864 218 4
-805 678 4
-488 197 2
-868 162 3
-510 313 5
-237 286 3
-588 71 4
-621 455 4
-445 1067 1
-24 324 5
-600 177 5
-655 572 2
-299 235 1
-194 783 2
-145 665 5
-673 327 4
-387 223 5
-798 306 3
-197 511 5
-498 1073 3
-659 483 4
-99 682 2
-64 2 3
-606 748 3
-64 748 1
-246 1089 1
-262 367 4
-650 88 3
-100 258 4
-34 294 1
-198 823 2
-322 507 4
-757 82 4
-655 654 3
-910 405 4
-145 939 4
-326 427 4
-708 764 4
-445 831 1
-583 708 5
-551 924 5
-653 172 3
-554 21 1
-425 1 2
-652 286 3
-682 358 3
-731 393 5
-77 168 4
-643 1065 4
-693 298 3
-496 1074 2
-407 85 4
-256 54 5
-339 212 4
-585 736 4
-125 369 3
-164 222 4
-608 131 4
-592 512 5
-654 720 4
-919 58 5
-295 69 5
-676 879 3
-254 417 3
-450 278 5
-7 655 5
-446 332 3
-187 65 5
-343 118 2
-748 118 2
-643 399 3
-316 180 4
-406 124 4
-11 504 3
-198 82 3
-838 190 4
-435 568 2
-331 268 5
-918 965 4
-308 483 3
-10 712 4
-498 483 3
-875 180 5
-883 226 3
-233 174 5
-707 485 4
-125 239 5
-115 969 1
-537 270 3
-249 169 5
-694 836 4
-624 980 4
-416 1089 2
-899 234 4
-642 393 5
-642 4 3
-913 227 1
-361 762 2
-577 728 3
-595 926 1
-601 153 4
-458 190 4
-577 385 5
-674 151 2
-471 418 3
-174 1139 2
-37 176 4
-870 268 3
-318 282 4
-186 12 1
-138 602 4
-332 293 4
-21 184 4
-109 739 4
-43 222 4
-489 688 2
-758 174 5
-258 877 3
-48 185 4
-450 493 4
-618 96 3
-94 77 3
-13 391 3
-506 42 3
-327 69 2
-838 286 4
-642 1 5
-697 881 2
-15 118 1
-264 179 5
-430 237 5
-181 147 1
-632 134 5
-216 396 3
-916 417 2
-190 930 2
-57 109 4
-724 351 1
-537 640 3
-867 651 5
-154 172 4
-488 239 4
-269 523 5
-425 670 3
-241 294 3
-146 311 4
-731 1503 5
-518 126 4
-104 13 3
-637 286 5
-91 328 4
-251 1014 5
-292 419 4
-270 288 5
-18 704 3
-327 249 2
-778 56 3
-887 96 4
-928 48 5
-922 204 3
-659 568 4
-642 15 5
-833 50 2
-358 482 2
-397 474 5
-864 559 4
-279 759 4
-406 787 3
-405 61 1
-640 750 5
-94 768 3
-823 502 5
-806 56 5
-889 721 3
-458 28 3
-253 4 4
-279 249 3
-60 603 5
-690 226 3
-905 100 4
-13 449 4
-665 475 3
-661 70 4
-715 111 3
-919 748 1
-343 727 4
-332 1 4
-130 286 5
-58 347 3
-843 550 3
-11 40 3
-398 58 4
-479 455 4
-537 70 4
-854 170 4
-71 50 3
-698 191 2
-461 9 5
-90 136 5
-26 322 3
-513 435 5
-537 1068 3
-329 174 4
-881 71 4
-894 1073 4
-280 499 4
-336 257 4
-887 427 5
-436 845 5
-854 1086 3
-292 11 5
-186 925 5
-803 242 5
-158 408 5
-109 215 3
-71 14 5
-42 239 5
-279 969 3
-186 306 4
-582 410 3
-130 294 5
-318 605 4
-66 300 5
-788 715 3
-843 635 2
-43 47 1
-57 181 5
-629 182 5
-234 73 2
-242 111 4
-663 1327 4
-932 1116 4
-625 476 2
-934 506 4
-32 122 2
-476 208 5
-903 25 4
-18 193 5
-383 124 4
-843 151 2
-63 412 3
-201 1169 4
-754 937 4
-457 243 2
-715 318 5
-177 693 4
-524 527 5
-45 237 4
-868 662 2
-207 712 4
-9 521 4
-663 655 4
-113 329 3
-942 500 5
-471 627 1
-469 1558 5
-178 597 4
-537 291 2
-536 176 3
-790 1047 3
-503 38 3
-334 762 3
-374 218 4
-731 56 2
-429 428 4
-577 550 3
-200 429 5
-24 64 5
-399 559 3
-303 327 1
-130 290 3
-934 972 3
-291 403 4
-459 222 4
-745 531 3
-493 124 3
-90 690 4
-437 748 4
-243 736 4
-881 566 4
-10 432 4
-460 327 4
-463 276 3
-733 124 5
-13 56 5
-443 340 5
-227 1068 4
-872 1376 2
-344 295 3
-864 225 3
-671 685 5
-660 228 3
-332 472 3
-181 424 1
-432 628 5
-906 121 4
-746 202 5
-223 117 5
-286 272 5
-307 655 4
-777 117 5
-261 687 5
-414 895 4
-941 273 3
-715 977 2
-308 215 3
-796 117 5
-40 1038 1
-234 768 2
-757 658 2
-561 93 4
-201 79 4
-296 313 5
-201 1224 2
-255 185 4
-19 294 3
-517 131 3
-519 351 5
-319 267 4
-313 195 5
-487 549 4
-677 742 4
-929 32 3
-529 880 4
-788 322 4
-21 443 4
-94 157 5
-697 250 4
-378 1221 3
-805 321 3
-281 258 2
-458 435 4
-327 215 4
-474 651 5
-62 697 4
-476 73 4
-535 848 3
-60 615 5
-174 197 5
-268 580 3
-542 367 4
-316 265 3
-158 1303 3
-345 126 5
-378 273 4
-303 405 4
-313 654 5
-164 405 5
-189 151 5
-195 55 4
-487 1209 4
-601 100 4
-412 182 4
-916 173 4
-794 19 4
-660 405 2
-655 865 4
-796 636 2
-144 73 3
-260 313 5
-393 1120 3
-31 1021 3
-334 449 3
-573 479 4
-293 685 3
-279 461 3
-478 196 3
-258 294 4
-664 588 3
-64 184 4
-23 8 4
-889 2 3
-396 148 4
-83 631 2
-577 202 4
-622 552 2
-896 768 2
-298 507 4
-378 191 5
-460 313 4
-91 1126 1
-608 736 4
-354 584 5
-423 344 4
-773 357 4
-537 837 3
-588 356 4
-70 172 5
-311 660 4
-606 180 4
-694 69 5
-194 22 5
-528 479 4
-591 283 4
-307 91 4
-450 1050 4
-336 845 1
-182 283 2
-334 47 4
-659 96 4
-634 137 3
-595 472 3
-698 505 2
-455 747 4
-867 156 5
-519 1238 5
-740 332 3
-508 176 4
-844 179 3
-727 88 5
-682 1084 2
-790 184 3
-878 740 2
-382 531 4
-57 304 5
-768 815 3
-887 934 4
-416 564 4
-889 469 4
-148 127 1
-348 1061 5
-610 505 4
-385 337 4
-744 628 2
-854 93 5
-665 172 4
-291 164 4
-11 707 5
-416 1160 4
-180 186 4
-405 736 5
-82 111 4
-705 815 3
-13 197 4
-533 109 2
-311 222 4
-393 239 4
-344 1172 4
-271 692 4
-222 102 2
-153 357 5
-311 86 5
-324 123 4
-932 131 4
-342 238 4
-472 477 5
-246 406 3
-144 1226 4
-276 1194 3
-642 686 5
-263 678 2
-711 1190 3
-521 474 3
-524 210 3
-349 847 4
-902 191 5
-344 451 4
-174 723 5
-233 249 5
-99 363 4
-391 286 4
-234 463 4
-5 17 4
-137 222 5
-223 329 2
-195 300 3
-653 1140 1
-739 197 1
-665 31 3
-722 928 3
-26 930 2
-506 95 5
-320 210 5
-699 1 3
-244 117 2
-899 479 4
-907 1028 5
-847 120 1
-792 1164 3
-236 79 4
-264 478 5
-655 516 2
-269 63 1
-459 932 4
-813 988 3
-786 82 4
-177 270 1
-92 947 4
-838 187 3
-536 190 5
-401 318 4
-660 210 4
-205 984 1
-116 1220 2
-159 289 2
-868 273 3
-727 651 3
-727 80 4
-736 181 2
-532 227 4
-790 41 3
-693 96 4
-501 111 3
-773 509 4
-472 423 5
-286 85 5
-838 275 5
-301 665 2
-193 100 5
-104 751 4
-882 95 4
-336 999 2
-400 328 3
-305 285 5
-802 674 2
-934 196 5
-545 194 3
-851 223 4
-102 38 2
-422 302 3
-174 364 1
-716 293 4
-788 451 4
-268 1 3
-805 9 3
-119 213 5
-253 331 3
-92 583 3
-13 590 2
-476 175 4
-378 942 3
-660 250 4
-854 281 3
-425 452 2
-184 277 3
-887 1278 2
-214 156 5
-210 219 3
-527 492 3
-539 155 4
-226 527 4
-43 91 3
-851 1016 5
-776 444 2
-889 237 4
-708 25 3
-207 684 3
-94 188 4
-832 307 4
-459 332 3
-293 447 4
-864 151 5
-694 228 4
-189 934 2
-757 743 2
-194 282 3
-709 636 3
-870 943 2
-293 1101 3
-761 201 2
-144 196 4
-655 619 3
-899 275 4
-197 22 5
-184 287 4
-254 167 3
-711 713 3
-452 86 4
-200 743 3
-665 588 4
-711 48 4
-217 226 1
-125 434 4
-543 83 4
-207 100 2
-250 237 2
-144 470 2
-838 143 5
-862 1109 5
-478 150 4
-345 1101 4
-621 686 5
-378 665 2
-201 219 4
-90 906 2
-284 754 3
-514 419 4
-839 257 3
-693 222 2
-545 474 3
-343 729 3
-327 4 4
-308 195 5
-90 493 5
-874 197 4
-536 435 3
-483 12 2
-572 9 5
-487 939 3
-234 1123 3
-379 435 5
-936 244 4
-825 286 4
-276 222 4
-758 68 3
-181 1302 1
-82 220 2
-943 566 4
-815 465 5
-503 319 3
-883 39 4
-823 50 5
-405 1573 1
-588 111 1
-106 64 4
-756 117 4
-757 1240 3
-91 323 2
-551 628 5
-666 651 5
-279 234 2
-122 661 4
-542 13 4
-95 392 3
-194 504 2
-933 121 3
-671 195 5
-929 144 3
-653 179 4
-385 383 1
-633 79 5
-853 879 4
-500 98 4
-7 201 2
-630 640 1
-254 548 2
-318 864 2
-466 98 3
-524 259 3
-639 194 4
-215 423 5
-303 1232 3
-774 1016 3
-334 221 5
-378 44 3
-621 940 3
-592 467 5
-533 283 3
-634 866 3
-619 121 5
-807 63 5
-174 381 5
-303 68 4
-394 739 4
-830 49 5
-29 303 4
-292 919 5
-766 294 2
-430 276 1
-393 189 4
-899 2 3
-109 559 3
-586 210 4
-862 1199 2
-44 313 4
-634 281 4
-293 286 3
-792 696 3
-643 1 5
-757 939 4
-354 190 4
-560 235 2
-48 522 2
-312 222 3
-636 9 3
-843 660 2
-334 204 4
-472 373 4
-339 144 3
-181 129 2
-833 1012 4
-160 228 2
-712 230 3
-152 371 4
-806 209 3
-207 529 4
-308 132 3
-454 942 2
-653 227 3
-276 423 5
-367 919 5
-331 305 5
-130 188 4
-695 286 3
-805 755 3
-863 683 1
-474 96 4
-864 476 2
-271 69 4
-378 317 5
-360 193 5
-151 451 5
-796 747 4
-109 159 4
-410 269 5
-189 170 4
-703 322 3
-247 1 4
-907 260 2
-457 239 5
-22 167 3
-67 1093 5
-313 136 5
-422 410 5
-244 291 2
-455 1086 3
-518 595 3
-468 97 5
-782 292 4
-533 554 1
-488 474 2
-122 46 5
-542 70 4
-666 656 4
-293 403 3
-253 427 5
-90 613 4
-586 576 3
-313 95 3
-234 606 5
-498 288 3
-442 53 3
-286 232 4
-933 117 2
-715 447 3
-194 509 3
-550 846 2
-766 613 3
-373 480 3
-393 507 2
-399 320 3
-773 431 1
-425 100 4
-13 132 4
-49 386 4
-764 191 3
-184 86 5
-918 213 5
-291 1471 3
-524 484 4
-271 523 4
-489 1280 3
-378 619 3
-30 435 5
-296 117 3
-217 183 3
-648 633 3
-886 42 5
-72 195 5
-859 928 3
-758 568 4
-94 50 5
-13 100 5
-643 739 3
-345 1074 3
-712 174 5
-773 89 4
-151 1 5
-830 399 5
-848 484 5
-303 54 3
-188 1263 3
-181 927 1
-806 407 3
-450 792 4
-642 1011 3
-109 29 3
-495 404 4
-553 132 4
-202 516 4
-559 257 3
-269 302 3
-130 946 4
-145 569 4
-671 210 5
-312 480 5
-455 214 3
-4 271 4
-450 490 5
-889 399 3
-708 1047 2
-197 720 2
-524 116 4
-275 96 3
-383 604 5
-627 1136 4
-892 88 4
-249 249 4
-682 196 5
-363 616 3
-234 288 3
-542 210 3
-585 463 5
-764 151 4
-305 129 3
-344 251 5
-276 451 3
-454 746 2
-880 394 3
-705 22 5
-537 12 3
-332 105 2
-181 1387 1
-588 554 3
-933 1028 2
-183 202 4
-497 200 3
-478 673 3
-351 990 5
-54 741 5
-346 1011 1
-122 673 3
-553 615 5
-805 21 2
-303 928 3
-885 405 4
-320 51 5
-543 62 3
-788 62 3
-393 1224 3
-450 11 5
-82 175 4
-479 338 1
-690 121 3
-867 690 5
-354 660 3
-367 443 4
-190 717 3
-73 683 2
-807 416 3
-524 1166 5
-786 699 4
-777 42 5
-577 284 4
-391 56 5
-331 160 5
-18 65 5
-53 258 4
-201 340 5
-556 178 5
-116 221 4
-532 1594 4
-699 1061 3
-908 701 4
-69 50 5
-383 528 4
-592 952 4
-679 154 4
-225 479 4
-567 135 3
-279 68 4
-76 531 4
-313 946 3
-370 173 3
-424 323 5
-109 172 5
-217 56 5
-299 136 4
-504 693 4
-424 115 1
-934 771 3
-840 169 5
-645 64 3
-586 451 4
-666 32 4
-7 226 5
-257 166 4
-920 288 3
-487 367 3
-393 944 4
-416 81 5
-793 294 5
-306 111 4
-472 123 4
-774 91 1
-450 566 4
-535 1149 4
-496 1133 3
-748 114 4
-386 685 4
-181 327 3
-246 541 3
-6 47 3
-533 724 4
-277 124 3
-456 13 4
-279 137 4
-643 7 4
-588 88 5
-537 352 1
-499 12 5
-790 1132 2
-514 473 3
-201 176 4
-95 511 4
-314 318 5
-89 739 2
-168 222 5
-514 87 5
-825 285 3
-303 369 1
-345 732 4
-173 332 4
-437 283 1
-710 751 3
-334 115 5
-736 286 4
-393 905 3
-416 11 4
-863 885 1
-851 4 5
-244 167 3
-918 972 5
-777 56 5
-536 213 5
-625 519 2
-606 468 4
-760 1135 4
-360 50 4
-56 176 5
-535 466 3
-666 1011 4
-753 98 5
-627 402 3
-744 1134 3
-45 284 4
-376 268 3
-854 83 4
-207 210 3
-747 196 2
-451 879 4
-230 385 1
-405 207 1
-334 476 3
-326 969 4
-593 233 2
-452 969 2
-655 1296 3
-79 515 5
-169 308 3
-624 123 3
-880 393 3
-881 234 3
-645 239 3
-42 82 4
-125 153 2
-320 173 5
-894 285 4
-804 229 4
-1 24 3
-60 613 4
-250 260 4
-708 278 4
-173 326 5
-425 79 4
-896 11 2
-696 1176 4
-889 696 3
-711 692 3
-174 384 1
-648 687 1
-833 861 3
-882 405 4
-932 148 2
-339 856 5
-272 654 5
-809 307 5
-196 306 4
-457 756 2
-634 508 4
-637 744 4
-506 199 4
-121 25 5
-222 810 2
-367 774 4
-280 1051 4
-429 39 3
-645 521 4
-298 127 5
-109 1245 2
-863 303 1
-689 151 3
-452 223 5
-644 117 4
-864 715 4
-891 181 3
-242 1011 3
-190 125 3
-389 820 3
-492 462 3
-715 658 4
-178 87 4
-22 210 3
-919 340 5
-144 48 5
-90 26 4
-850 172 5
-276 79 4
-731 168 1
-592 469 4
-308 510 3
-164 742 5
-314 627 4
-387 735 2
-682 1231 2
-256 216 5
-486 117 3
-500 8 4
-642 369 2
-881 663 5
-506 755 4
-365 342 2
-617 453 1
-160 763 4
-276 1052 2
-552 126 4
-454 474 4
-181 713 2
-13 765 2
-429 735 4
-440 886 5
-15 924 3
-269 664 5
-399 238 1
-162 597 4
-94 556 3
-387 718 4
-655 196 3
-417 102 3
-52 845 5
-691 735 5
-932 562 2
-303 1315 3
-498 77 2
-92 318 2
-486 244 3
-710 504 4
-817 124 4
-603 62 2
-120 258 5
-693 546 1
-158 583 3
-903 1142 5
-12 196 5
-504 323 4
-350 589 5
-314 1085 1
-6 536 4
-493 121 5
-27 148 3
-455 56 5
-556 294 2
-454 88 4
-724 358 1
-537 633 3
-930 238 4
-535 919 4
-624 312 4
-796 500 4
-863 299 2
-405 526 1
-393 317 4
-936 287 4
-497 97 4
-151 393 2
-407 211 4
-395 286 4
-829 124 4
-454 356 1
-141 1142 1
-796 69 5
-633 5 3
-653 239 5
-711 143 5
-607 528 4
-384 286 4
-823 531 4
-297 223 5
-484 195 5
-378 62 4
-316 64 4
-724 948 1
-393 399 4
-462 678 3
-380 518 3
-624 273 4
-4 361 5
-704 347 4
-346 293 3
-704 662 3
-345 93 4
-734 275 4
-95 631 4
-158 546 3
-733 20 5
-668 993 4
-655 1149 3
-342 732 3
-283 1114 5
-446 300 3
-286 403 5
-318 100 5
-746 24 4
-913 596 1
-642 258 3
-826 336 4
-645 319 3
-884 146 3
-399 454 3
-312 496 5
-715 470 4
-622 156 5
-314 682 5
-416 282 5
-174 138 1
-75 427 4
-460 676 4
-268 1046 3
-13 343 1
-804 456 3
-405 469 1
-151 202 5
-432 257 5
-739 749 5
-580 825 4
-268 386 2
-911 627 3
-311 365 4
-650 391 2
-401 143 4
-22 208 5
-903 111 3
-459 295 3
-828 6 1
-355 358 4
-70 482 4
-762 246 1
-450 923 5
-92 239 4
-763 159 3
-658 518 4
-234 234 4
-881 62 4
-279 166 4
-435 218 3
-484 778 5
-294 827 1
-436 568 5
-509 268 2
-907 245 4
-393 597 3
-758 1025 3
-387 10 4
-293 1132 3
-82 523 5
-592 12 5
-748 188 4
-758 746 4
-142 124 4
-660 890 1
-664 513 4
-450 23 5
-497 63 3
-145 96 5
-385 30 5
-303 368 1
-698 100 2
-179 893 2
-715 1045 2
-739 498 4
-804 447 3
-1 144 4
-684 186 4
-451 259 4
-295 461 5
-821 495 5
-783 876 4
-60 70 4
-343 159 2
-606 197 3
-710 418 3
-135 233 3
-76 1159 3
-507 300 5
-385 1252 5
-277 925 4
-566 191 4
-852 7 3
-59 55 5
-908 318 5
-320 692 4
-201 408 4
-152 67 5
-284 903 4
-399 383 2
-863 322 1
-426 1116 4
-565 30 5
-235 462 3
-867 294 3
-37 210 4
-886 17 4
-508 216 5
-660 87 2
-579 520 4
-871 197 3
-933 67 1
-214 79 4
-450 966 4
-7 472 2
-749 498 4
-645 177 4
-896 27 1
-293 5 3
-227 411 4
-439 591 4
-514 269 4
-186 983 3
-708 412 1
-49 347 3
-13 868 5
-422 448 4
-278 294 4
-145 761 4
-234 176 3
-942 478 5
-416 1336 1
-450 506 5
-622 418 3
-815 98 4
-23 694 4
-682 1655 2
-896 802 2
-905 748 2
-622 1419 2
-885 142 2
-407 169 5
-871 326 5
-501 150 5
-427 1265 5
-116 311 3
-890 654 5
-90 904 3
-251 181 4
-311 520 5
-776 23 4
-632 156 3
-303 230 3
-758 316 5
-303 176 5
-898 288 4
-325 469 4
-642 398 2
-525 15 4
-450 170 5
-22 636 3
-364 286 5
-18 151 3
-661 313 4
-896 720 1
-660 281 3
-276 425 4
-405 529 1
-26 109 3
-576 237 4
-727 1437 2
-345 44 3
-735 676 3
-669 474 4
-399 94 5
-659 705 5
-536 181 5
-91 28 4
-829 125 3
-415 328 5
-901 451 4
-465 835 3
-435 1069 4
-407 71 3
-833 217 2
-82 826 3
-393 1047 3
-746 222 3
-407 161 2
-914 775 3
-751 386 3
-916 474 4
-151 125 4
-588 1508 3
-523 14 5
-464 479 4
-113 300 3
-517 294 1
-521 204 4
-548 100 5
-60 327 4
-15 476 4
-863 329 2
-58 240 4
-660 307 3
-585 1193 5
-825 988 3
-862 479 4
-194 181 3
-721 319 3
-347 132 5
-399 157 3
-7 187 4
-442 129 4
-879 282 4
-854 188 4
-436 21 3
-336 388 1
-942 97 5
-479 202 4
-405 1479 1
-561 379 2
-90 382 5
-721 1039 5
-453 367 2
-456 959 4
-538 202 4
-344 151 5
-293 187 3
-189 527 5
-76 772 3
-26 685 3
-424 258 2
-593 1 3
-119 718 5
-463 225 3
-401 357 4
-840 747 4
-853 887 2
-240 289 4
-297 471 3
-18 52 5
-102 307 4
-804 411 3
-59 591 4
-563 181 4
-901 161 5
-894 270 3
-655 517 4
-374 568 5
-279 216 3
-840 484 5
-311 22 4
-880 268 5
-849 288 5
-613 303 4
-178 98 5
-95 175 5
-289 151 2
-592 336 1
-440 304 5
-109 118 3
-592 1022 5
-889 566 3
-371 79 5
-872 278 3
-886 1217 4
-748 271 3
-342 558 5
-276 21 3
-798 671 2
-885 815 4
-22 449 1
-840 134 3
-929 135 5
-432 864 2
-609 15 5
-158 293 4
-459 685 3
-137 79 5
-463 508 4
-783 307 5
-193 96 1
-55 1016 1
-94 293 4
-345 498 4
-346 712 3
-416 310 5
-801 326 4
-682 201 4
-110 642 2
-334 716 3
-804 23 4
-354 485 4
-128 229 2
-64 496 5
-336 546 3
-454 478 2
-718 222 4
-486 327 3
-94 588 4
-595 294 2
-62 191 5
-158 686 5
-82 174 5
-395 365 5
-267 545 2
-22 878 1
-916 28 4
-536 487 4
-279 831 5
-601 133 4
-59 679 4
-749 56 2
-93 866 2
-655 57 3
-279 425 4
-256 692 5
-741 945 5
-901 795 3
-772 328 5
-234 10 3
-747 87 5
-449 753 5
-666 385 3
-43 123 1
-474 244 4
-83 476 3
-486 322 2
-663 129 3
-751 689 2
-660 1133 2
-541 181 5
-472 29 5
-886 919 4
-864 393 3
-501 25 3
-293 510 3
-256 554 4
-297 137 5
-676 259 4
-41 265 3
-373 117 4
-714 1152 2
-1 190 5
-695 312 3
-711 121 1
-416 71 4
-868 1076 1
-174 1313 4
-933 578 1
-564 289 4
-545 97 3
-766 435 3
-600 1231 2
-894 306 4
-806 156 4
-772 288 2
-537 1111 3
-574 258 5
-712 781 4
-622 480 4
-554 696 3
-716 225 3
-551 227 5
-338 517 5
-727 123 3
-453 385 3
-42 175 2
-194 616 3
-473 319 3
-125 134 5
-593 173 5
-109 392 3
-293 464 3
-189 568 4
-78 298 3
-660 252 2
-671 4 5
-435 751 4
-293 558 3
-619 79 5
-407 930 2
-622 196 3
-650 219 3
-474 471 3
-546 567 4
-938 1061 4
-715 12 4
-149 303 4
-766 71 3
-745 275 1
-15 237 3
-504 773 3
-770 410 4
-514 215 4
-663 268 3
-906 9 4
-268 379 1
-504 559 5
-551 202 4
-916 674 3
-868 1240 5
-796 210 3
-132 251 4
-447 278 3
-448 268 3
-290 234 3
-940 8 5
-537 404 3
-306 100 4
-18 958 5
-95 117 4
-579 523 3
-493 191 4
-62 81 4
-416 28 5
-262 68 2
-197 306 2
-307 527 5
-249 239 3
-877 288 3
-207 154 2
-82 582 4
-894 256 3
-494 143 5
-456 60 4
-222 7 5
-430 302 4
-18 153 4
-664 367 3
-716 620 3
-774 202 5
-378 1267 3
-303 416 3
-928 246 5
-291 825 4
-639 739 3
-99 196 4
-823 25 3
-11 715 3
-404 683 4
-151 169 5
-49 904 2
-543 85 2
-886 160 1
-708 269 3
-626 328 1
-374 55 2
-22 712 4
-715 202 5
-468 318 5
-197 665 4
-179 353 1
-655 1045 3
-883 506 5
-627 64 5
-200 112 3
-276 1047 3
-174 953 5
-458 969 4
-507 1089 5
-444 1483 2
-334 98 4
-788 620 3
-374 642 1
-187 732 3
-622 1552 2
-371 24 4
-554 172 5
-317 355 4
-497 121 4
-38 140 5
-893 820 3
-715 79 5
-5 408 5
-398 735 4
-506 191 4
-332 984 2
-291 558 4
-297 479 5
-659 4 3
-296 963 5
-653 226 3
-561 639 3
-782 1588 3
-916 238 4
-303 867 3
-213 258 4
-387 642 4
-189 166 4
-783 880 4
-877 582 2
-712 195 3
-339 9 5
-493 925 3
-793 1365 2
-699 479 3
-94 997 4
-152 153 4
-16 152 4
-620 565 4
-682 556 2
-450 195 4
-90 9 4
-854 12 5
-654 431 4
-597 988 1
-7 29 3
-3 335 1
-748 69 4
-506 391 2
-537 937 3
-391 61 5
-653 581 1
-279 284 1
-62 3 3
-934 624 4
-435 153 3
-16 509 2
-805 476 1
-648 436 5
-495 62 3
-109 177 4
-6 71 4
-897 11 2
-741 696 3
-735 100 2
-363 451 2
-194 550 3
-391 8 3
-665 298 3
-872 300 5
-359 408 5
-7 178 4
-211 69 3
-568 482 4
-627 195 4
-566 385 3
-89 277 4
-363 313 5
-72 228 1
-481 484 4
-181 1151 1
-97 466 3
-714 258 4
-457 284 3
-796 181 5
-482 988 4
-59 56 5
-533 31 3
-848 71 5
-416 1221 5
-804 554 2
-339 737 3
-600 449 4
-170 678 4
-203 50 5
-582 919 5
-899 177 3
-865 845 1
-164 276 3
-117 1012 4
-659 179 1
-797 259 3
-223 1150 2
-845 904 3
-709 431 5
-308 19 3
-807 527 5
-283 659 5
-823 408 5
-919 458 2
-323 508 4
-361 531 5
-295 161 4
-888 792 5
-593 159 4
-213 69 3
-280 274 5
-625 214 4
-897 73 3
-650 735 3
-823 55 4
-6 213 4
-579 234 3
-812 302 3
-498 12 4
-347 763 5
-524 521 4
-451 457 2
-682 834 3
-303 31 3
-392 650 5
-729 751 3
-557 529 5
-59 109 4
-553 589 5
-184 629 3
-89 181 4
-883 154 4
-291 575 2
-250 151 4
-450 710 3
-425 823 3
-807 511 5
-864 195 4
-83 411 2
-716 495 4
-688 329 5
-276 346 4
-889 408 3
-669 187 5
-292 169 5
-534 109 4
-291 12 5
-110 31 3
-60 69 4
-58 204 4
-872 682 3
-293 70 3
-712 1469 4
-499 69 5
-163 64 4
-423 127 4
-828 83 3
-346 55 5
-321 175 3
-923 763 4
-782 1589 3
-835 708 5
-650 523 3
-496 1060 1
-215 183 5
-201 381 3
-437 387 2
-269 1444 1
-214 182 4
-184 89 4
-158 124 4
-804 97 4
-843 102 2
-580 123 4
-347 1016 3
-743 311 5
-396 471 4
-926 302 4
-49 475 4
-301 164 3
-293 1048 3
-660 208 4
-894 1258 3
-438 619 4
-712 1480 4
-435 696 3
-59 32 4
-521 291 1
-69 109 3
-897 141 4
-387 52 5
-894 248 4
-178 511 5
-942 117 4
-659 157 4
-679 56 4
-15 251 2
-880 233 4
-901 168 4
-230 153 5
-234 520 4
-496 378 1
-615 427 5
-521 1012 3
-201 51 2
-144 180 4
-848 25 5
-72 25 5
-722 237 4
-360 308 4
-500 50 3
-816 271 4
-165 127 4
-835 135 5
-180 318 5
-781 50 5
-279 21 3
-525 123 3
-262 406 3
-655 162 3
-715 168 4
-41 97 3
-684 365 4
-553 99 5
-782 1652 1
-178 226 4
-442 11 4
-592 971 4
-682 67 4
-347 286 3
-7 515 3
-195 181 5
-236 216 5
-798 740 2
-630 983 3
-839 742 3
-564 127 4
-714 255 2
-450 110 4
-938 1014 4
-7 430 3
-409 429 5
-178 143 4
-344 313 3
-293 294 2
-452 290 2
-385 492 2
-916 1119 3
-751 101 4
-51 83 5
-813 326 3
-177 64 4
-710 202 3
-128 204 4
-294 299 3
-52 919 5
-655 581 2
-115 496 1
-586 763 4
-599 1095 4
-111 354 4
-6 484 5
-936 327 4
-608 288 5
-97 133 1
-885 662 3
-916 193 4
-508 52 4
-246 117 3
-764 71 5
-62 712 4
-69 258 4
-456 660 5
-881 542 1
-85 153 3
-831 316 3
-385 705 3
-25 427 4
-13 855 4
-344 487 5
-95 178 5
-303 1270 1
-373 386 3
-205 326 4
-682 234 3
-313 177 4
-312 1 5
-488 83 4
-474 89 5
-727 1217 3
-933 834 1
-95 539 4
-234 473 5
-721 199 4
-59 1028 1
-601 230 4
-682 195 4
-91 127 5
-392 165 5
-201 20 2
-188 164 4
-409 22 2
-69 307 2
-429 1209 3
-457 1168 5
-380 181 3
-621 200 4
-764 174 5
-815 483 5
-114 210 3
-405 761 1
-767 524 5
-620 393 5
-393 9 4
-389 416 4
-254 50 5
-716 1203 2
-630 70 2
-919 1014 4
-347 427 4
-871 1119 3
-846 396 5
-92 540 2
-121 294 4
-102 559 3
-456 955 4
-764 140 3
-467 181 3
-883 209 3
-618 731 2
-459 250 5
-381 480 5
-665 748 4
-807 8 4
-210 527 5
-10 69 4
-509 892 1
-805 413 2
-886 813 4
-160 497 4
-560 1134 3
-181 1393 1
-634 760 3
-215 88 3
-344 86 4
-493 127 3
-334 257 4
-894 25 2
-323 22 5
-857 475 5
-327 1098 4
-181 681 1
-472 252 4
-429 483 5
-72 121 3
-189 1056 3
-423 340 4
-193 554 3
-266 245 1
-405 356 5
-405 2 1
-32 455 2
-181 938 1
-664 212 4
-749 216 4
-311 348 4
-869 596 3
-7 560 3
-85 70 4
-710 79 4
-943 184 5
-354 197 4
-378 939 4
-184 568 2
-481 100 4
-234 9 3
-727 441 2
-882 135 5
-92 527 3
-268 1228 1
-360 933 3
-268 1073 4
-468 238 3
-537 243 1
-128 71 4
-934 1135 3
-830 205 5
-429 1217 2
-77 199 5
-11 211 3
-725 301 4
-663 685 4
-886 33 4
-269 53 1
-593 118 4
-643 630 3
-870 655 4
-168 988 2
-450 185 5
-827 690 3
-307 209 5
-442 33 3
-916 4 4
-468 150 5
-279 554 1
-756 554 1
-313 414 3
-365 908 3
-660 68 4
-213 79 5
-363 380 4
-186 385 4
-234 785 3
-102 273 3
-276 78 4
-788 53 1
-321 64 3
-847 88 2
-846 161 4
-429 410 4
-776 656 5
-861 584 5
-722 151 5
-561 526 3
-234 205 3
-889 474 4
-881 732 5
-805 386 3
-178 233 4
-390 304 5
-758 230 4
-773 386 3
-833 1353 3
-531 327 3
-873 300 4
-493 265 5
-393 302 4
-784 323 4
-541 91 5
-328 692 4
-399 554 3
-748 181 4
-899 125 3
-643 24 4
-373 215 4
-521 90 2
-714 276 2
-827 333 3
-666 428 3
-181 1094 1
-303 491 4
-308 58 3
-682 1093 3
-28 89 4
-637 1226 2
-847 195 4
-913 172 5
-435 204 3
-896 281 2
-249 250 4
-389 618 4
-894 818 3
-429 583 3
-758 305 4
-321 483 5
-655 921 3
-940 204 4
-882 50 5
-445 902 4
-684 520 4
-267 1035 4
-394 405 3
-262 358 3
-648 225 1
-533 77 4
-301 128 5
-21 441 3
-864 190 4
-563 1035 4
-450 704 3
-280 393 4
-346 2 5
-514 153 4
-458 174 3
-869 411 4
-177 960 3
-104 311 1
-430 328 4
-796 477 2
-298 205 5
-450 87 5
-327 1170 4
-621 384 3
-95 239 3
-405 397 4
-154 286 4
-263 1444 3
-556 288 4
-143 258 3
-648 526 3
-457 121 4
-846 231 2
-919 447 4
-328 50 4
-307 56 4
-436 928 4
-882 28 5
-642 843 3
-727 566 3
-151 470 3
-52 302 4
-287 346 5
-276 678 3
-445 79 4
-508 735 4
-229 302 5
-701 19 5
-35 321 3
-862 198 5
-326 657 5
-904 709 3
-560 122 3
-172 612 3
-892 380 4
-332 9 4
-533 295 4
-699 23 4
-367 183 5
-452 48 5
-542 1059 4
-659 792 4
-354 962 4
-374 25 5
-846 499 4
-493 890 3
-145 515 5
-892 73 3
-478 222 2
-541 405 3
-533 845 4
-605 340 4
-758 554 3
-506 749 4
-770 14 5
-665 185 4
-650 135 4
-752 678 3
-64 156 4
-224 980 1
-203 117 4
-181 1323 1
-545 751 3
-301 824 3
-417 135 3
-618 1225 2
-11 51 4
-889 249 3
-176 181 3
-521 721 4
-200 478 5
-425 144 4
-13 603 4
-752 752 3
-274 476 4
-23 404 4
-214 175 5
-707 462 4
-864 123 4
-194 712 3
-851 748 3
-342 319 4
-21 263 1
-118 175 5
-347 713 3
-524 837 2
-899 568 4
-123 514 5
-497 128 4
-399 53 4
-917 328 2
-880 186 4
-455 372 4
-663 710 3
-136 9 5
-854 483 4
-880 1185 1
-315 186 4
-479 546 2
-37 831 2
-176 272 5
-262 338 4
-489 1265 2
-450 589 3
-716 504 5
-256 120 1
-56 450 3
-655 402 2
-639 191 3
-293 238 4
-326 565 3
-733 149 4
-883 199 4
-655 93 3
-436 447 1
-271 506 4
-870 238 4
-328 349 2
-109 180 3
-311 83 5
-13 280 4
-424 289 5
-934 177 3
-25 498 4
-555 235 3
-301 219 4
-823 56 5
-409 527 4
-753 215 5
-334 642 5
-552 1315 3
-860 339 3
-328 849 3
-276 415 3
-497 232 3
-109 186 3
-845 346 3
-236 225 3
-94 1032 2
-642 419 4
-665 357 4
-344 408 5
-533 685 4
-659 469 4
-151 487 5
-694 211 5
-174 1 3
-592 844 4
-399 118 3
-846 28 5
-786 265 4
-682 215 4
-474 685 3
-426 492 5
-387 1129 4
-455 291 3
-660 215 3
-459 676 3
-320 274 4
-407 216 4
-655 283 3
-892 482 5
-293 804 1
-379 451 4
-149 269 5
-551 721 5
-111 340 4
-650 27 3
-560 127 5
-694 393 3
-598 691 2
-868 127 4
-861 45 5
-774 563 1
-776 674 3
-474 435 5
-221 566 3
-7 603 4
-592 134 5
-525 276 5
-849 38 5
-928 173 4
-279 393 1
-894 898 4
-606 756 3
-881 678 2
-343 26 3
-75 1 4
-468 126 3
-280 2 3
-49 334 4
-619 188 4
-508 511 4
-727 544 3
-653 193 4
-64 172 4
-535 631 5
-42 1050 3
-653 1206 3
-796 89 5
-116 180 5
-527 317 4
-634 628 4
-144 528 4
-130 1095 3
-790 417 2
-864 549 3
-486 872 5
-640 342 5
-487 781 3
-336 722 3
-500 846 3
-916 569 2
-393 249 3
-201 563 1
-846 748 3
-825 116 3
-301 269 5
-435 118 2
-932 489 4
-450 934 3
-730 748 4
-877 584 4
-328 559 3
-933 1246 1
-249 172 3
-271 9 4
-145 728 2
-276 470 3
-426 519 4
-102 231 2
-49 151 5
-454 97 4
-484 313 5
-467 1017 2
-328 778 3
-504 632 3
-269 793 4
-268 302 5
-425 338 1
-314 7 4
-683 609 3
-670 479 5
-71 64 4
-224 977 2
-506 328 4
-805 367 4
-223 237 5
-23 203 4
-630 276 1
-357 472 3
-656 344 4
-537 735 3
-654 300 5
-798 755 3
-122 956 4
-257 151 4
-104 258 3
-51 496 4
-865 108 1
-102 476 3
-634 322 3
-5 200 2
-655 46 4
-328 905 3
-790 364 2
-695 995 4
-457 285 5
-487 232 4
-307 195 3
-317 683 2
-744 301 3
-918 971 4
-308 546 3
-305 251 5
-339 65 4
-92 15 3
-123 23 4
-13 914 2
-512 23 4
-600 431 3
-505 271 4
-21 129 4
-244 393 3
-15 685 4
-13 573 3
-130 150 5
-173 301 5
-426 651 4
-251 55 3
-363 67 1
-72 97 4
-343 405 4
-833 1143 4
-520 315 4
-671 144 4
-201 268 4
-846 12 5
-554 939 4
-536 1063 5
-684 210 3
-497 570 3
-815 639 2
-181 874 1
-417 568 2
-90 708 5
-405 785 1
-161 316 5
-76 806 4
-618 955 2
-378 25 4
-380 631 4
-380 194 4
-877 286 2
-196 382 4
-527 175 3
-640 161 4
-181 832 1
-877 61 5
-167 136 4
-787 310 5
-696 311 5
-533 580 3
-144 144 4
-645 209 5
-3 288 2
-596 295 4
-313 586 2
-94 1221 3
-8 22 5
-18 504 5
-881 233 3
-736 296 4
-346 7 2
-637 257 2
-374 116 1
-313 608 4
-804 435 3
-378 716 3
-532 769 2
-200 431 5
-268 622 3
-898 302 4
-318 381 1
-184 259 3
-223 873 3
-268 558 3
-412 70 4
-342 606 5
-236 273 1
-731 215 5
-648 435 5
-838 83 5
-837 328 4
-421 709 4
-481 238 4
-566 218 4
-276 871 2
-805 140 3
-221 204 4
-860 204 4
-711 652 4
-126 245 3
-436 715 4
-59 428 5
-97 32 5
-437 699 4
-292 2 4
-588 121 5
-303 1040 1
-318 58 4
-718 121 4
-90 89 5
-551 790 2
-313 178 5
-567 604 4
-416 53 2
-450 25 3
-608 195 1
-622 432 5
-721 321 3
-618 215 4
-385 235 5
-162 50 5
-831 117 3
-543 157 3
-936 813 5
-603 294 4
-72 423 5
-125 269 1
-655 794 1
-615 644 4
-11 726 3
-434 1197 5
-474 187 5
-490 1012 3
-913 475 4
-618 588 4
-453 427 3
-552 322 3
-453 652 3
-666 1021 5
-614 546 1
-831 328 3
-623 815 2
-751 21 5
-881 943 4
-933 735 3
-279 823 3
-417 324 1
-188 356 4
-416 333 4
-28 443 4
-327 431 3
-7 624 4
-648 111 5
-493 181 5
-788 621 3
-378 370 2
-497 771 4
-283 288 2
-552 932 3
-332 895 5
-434 628 1
-18 524 4
-233 313 5
-96 144 4
-338 100 4
-56 728 3
-420 283 5
-474 520 5
-830 97 4
-299 151 4
-506 494 5
-639 215 1
-151 775 2
-117 751 5
-370 134 4
-447 123 3
-4 324 5
-643 161 3
-917 535 4
-239 516 5
-249 7 5
-292 657 5
-774 168 1
-658 55 4
-405 702 1
-431 245 4
-406 89 4
-245 300 4
-386 455 3
-276 823 3
-510 245 3
-805 108 3
-618 419 4
-896 100 3
-244 744 3
-595 880 3
-425 358 4
-259 210 4
-184 529 4
-630 120 4
-768 756 3
-615 517 5
-733 1067 5
-493 65 4
-116 257 3
-72 530 4
-819 1160 4
-773 1071 2
-10 703 5
-391 132 4
-305 176 4
-234 604 5
-872 294 3
-758 628 4
-457 252 4
-834 287 2
-23 504 4
-940 12 4
-188 28 3
-881 561 4
-655 150 3
-374 761 3
-43 926 2
-56 265 4
-486 515 5
-45 1 5
-416 253 3
-368 379 4
-677 91 5
-908 151 3
-115 466 5
-524 573 4
-417 411 2
-703 7 4
-293 877 2
-790 716 4
-406 220 3
-751 385 4
-2 299 4
-13 349 1
-788 153 3
-804 1260 3
-121 742 5
-86 326 3
-314 280 3
-369 268 5
-660 391 2
-663 763 5
-643 98 3
-354 257 3
-772 323 4
-830 523 4
-545 569 3
-184 715 4
-626 748 2
-499 275 3
-798 399 5
-311 51 4
-913 83 4
-416 14 4
-505 495 3
-7 593 5
-733 1658 3
-378 1107 3
-297 465 3
-672 127 4
-279 153 5
-523 186 3
-334 1073 4
-498 474 4
-674 257 4
-314 609 4
-437 393 3
-85 520 3
-15 20 3
-939 742 5
-606 11 5
-87 70 5
-910 25 3
-405 1266 1
-185 114 4
-104 122 3
-314 94 4
-881 864 3
-7 642 3
-758 203 5
-537 194 3
-148 473 5
-486 124 5
-840 215 4
-632 549 3
-524 578 5
-375 684 4
-717 280 4
-870 87 5
-141 1023 4
-854 482 3
-468 96 5
-183 50 2
-490 993 1
-416 1051 3
-787 306 3
-56 79 4
-425 168 5
-823 144 5
-887 1029 5
-92 405 2
-429 430 4
-450 191 5
-223 620 2
-134 316 4
-3 352 2
-513 222 5
-38 252 5
-276 800 3
-738 697 2
-295 50 5
-385 1462 4
-735 127 4
-537 478 4
-488 269 3
-214 527 4
-149 272 3
-303 153 5
-910 414 4
-379 271 3
-815 650 2
-221 64 5
-56 372 3
-693 12 4
-820 264 3
-608 196 5
-554 328 4
-394 230 3
-660 132 3
-580 147 3
-110 802 3
-525 1047 2
-648 1033 2
-413 270 4
-59 258 3
-445 281 1
-313 228 3
-694 1269 5
-893 172 5
-455 582 2
-308 671 4
-599 888 5
-716 132 5
-280 629 4
-338 310 3
-655 1465 2
-171 306 3
-940 4 2
-101 763 3
-210 94 4
-818 316 4
-532 402 5
-282 890 4
-566 108 2
-393 36 3
-334 502 3
-750 873 3
-313 79 5
-932 507 5
-21 50 3
-454 318 5
-200 318 5
-788 120 2
-709 781 3
-892 768 4
-655 218 3
-234 531 3
-7 145 1
-339 410 2
-389 471 4
-332 240 4
-46 294 2
-585 1347 2
-614 286 2
-62 655 3
-650 309 3
-653 157 5
-291 235 2
-564 1399 2
-892 625 3
-804 151 3
-682 659 1
-476 451 3
-843 403 2
-357 304 5
-643 229 3
-23 145 3
-561 24 3
-861 242 5
-697 225 3
-312 173 3
-637 405 1
-831 354 4
-566 25 2
-234 197 5
-908 174 3
-814 413 2
-411 228 3
-145 240 5
-89 283 4
-660 290 4
-600 1407 2
-423 1238 3
-757 210 4
-457 527 5
-92 234 4
-940 651 4
-339 159 3
-550 257 4
-743 408 4
-588 268 5
-418 1313 2
-85 705 5
-561 443 4
-414 301 3
-286 339 5
-542 246 3
-699 319 3
-635 748 2
-764 472 3
-28 234 4
-535 56 3
-429 235 3
-10 13 3
-299 185 3
-125 235 2
-757 38 3
-709 22 5
-337 106 2
-896 371 2
-868 211 3
-561 475 3
-7 163 4
-380 515 4
-459 125 4
-769 222 4
-823 161 3
-896 288 3
-321 663 2
-44 197 4
-48 690 4
-179 271 1
-518 124 3
-796 183 5
-655 726 2
-26 508 3
-433 1005 5
-222 949 3
-407 525 4
-826 511 3
-383 205 4
-416 721 3
-413 327 3
-125 201 3
-790 451 3
-572 476 4
-312 512 3
-320 586 3
-72 461 3
-226 24 4
-198 518 3
-279 450 4
-870 288 4
-270 26 5
-466 326 3
-405 1588 1
-464 1226 4
-637 831 1
-883 345 3
-854 86 3
-232 246 4
-59 210 4
-13 794 4
-703 147 3
-113 268 4
-896 1240 4
-666 632 4
-270 250 2
-851 161 3
-847 1031 2
-532 588 5
-7 620 4
-547 316 5
-409 172 5
-684 408 5
-678 287 3
-271 527 5
-793 696 3
-342 518 3
-280 619 4
-882 101 3
-692 127 3
-119 930 3
-311 510 4
-655 508 3
-864 81 3
-617 674 3
-506 761 2
-633 82 4
-405 1471 1
-447 597 3
-805 181 3
-659 498 3
-116 840 1
-764 13 2
-835 514 3
-932 177 4
-49 569 3
-493 1088 2
-727 747 2
-280 162 3
-158 298 3
-770 289 5
-33 751 4
-733 121 3
-770 875 4
-284 319 3
-279 1242 1
-535 632 4
-155 306 5
-193 294 1
-733 146 3
-838 12 4
-569 405 3
-899 69 3
-370 484 4
-401 204 5
-836 523 5
-449 61 5
-161 132 1
-933 449 1
-698 404 1
-11 97 4
-388 121 4
-499 690 4
-106 196 5
-152 527 4
-526 475 5
-94 587 4
-303 979 4
-916 721 4
-453 164 3
-707 811 4
-347 240 5
-216 15 3
-21 243 2
-59 611 3
-763 283 4
-279 649 3
-398 228 5
-30 531 5
-450 176 4
-472 540 3
-750 881 2
-269 961 5
-464 678 3
-98 25 5
-276 323 3
-145 454 1
-398 654 4
-230 1 5
-312 8 5
-63 546 2
-295 624 5
-206 359 1
-547 751 4
-704 340 3
-376 197 4
-184 393 4
-637 293 3
-887 183 1
-848 498 5
-786 458 3
-82 496 4
-638 210 4
-271 625 3
-927 410 1
-310 258 3
-740 294 4
-276 455 4
-230 451 4
-166 243 3
-346 180 5
-385 201 4
-319 333 4
-5 403 3
-311 1041 3
-500 39 4
-85 474 5
-436 1135 4
-564 924 3
-75 678 3
-82 289 1
-796 125 4
-748 204 3
-234 506 4
-898 327 5
-189 97 4
-468 642 3
-474 88 4
-204 333 1
-152 1041 5
-276 271 4
-450 469 4
-936 124 4
-854 153 4
-747 268 5
-526 302 5
-886 1209 2
-52 657 5
-399 1217 4
-627 174 3
-592 483 5
-655 212 3
-877 302 2
-569 125 3
-336 1098 3
-774 1118 3
-637 595 3
-639 1005 2
-59 123 3
-254 679 2
-90 1199 5
-548 642 4
-332 471 4
-592 1166 3
-114 855 3
-731 478 4
-733 534 3
-770 1 5
-271 300 2
-619 326 2
-139 1233 5
-751 316 4
-203 815 4
-693 685 4
-212 515 4
-280 135 4
-837 181 3
-763 25 4
-239 498 4
-806 856 5
-621 746 4
-830 427 5
-916 931 1
-537 178 4
-187 215 3
-363 347 3
-654 568 4
-538 127 5
-16 144 5
-497 3 4
-384 689 4
-721 326 4
-406 526 5
-48 496 5
-451 1394 1
-524 79 4
-798 1089 3
-378 182 4
-790 597 3
-22 999 4
-914 692 3
-255 323 2
-285 319 3
-328 260 2
-683 906 4
-18 95 4
-543 947 4
-194 449 1
-406 404 5
-458 387 4
-13 25 1
-642 559 5
-664 678 2
-703 1197 3
-66 741 4
-119 40 4
-471 404 2
-450 969 4
-858 292 3
-766 52 4
-222 723 3
-72 5 4
-766 238 4
-134 15 5
-184 639 3
-894 295 3
-495 1135 5
-416 475 2
-405 1069 1
-735 285 4
-311 732 4
-320 284 4
-894 479 5
-523 70 5
-896 85 3
-107 321 2
-687 313 5
-552 742 4
-457 28 5
-274 275 5
-405 420 5
-189 16 3
-480 50 4
-95 1230 1
-26 323 2
-504 728 3
-298 79 5
-845 900 3
-457 51 5
-1 267 4
-458 98 3
-937 295 4
-403 748 5
-151 1017 2
-295 72 4
-546 447 3
-243 317 5
-539 131 4
-497 33 4
-291 129 5
-7 543 3
-336 591 5
-703 323 2
-653 180 5
-221 943 4
-102 98 4
-358 65 4
-56 781 4
-311 402 4
-637 515 4
-213 176 4
-405 1222 1
-896 504 3
-502 338 4
-682 190 4
-624 270 3
-500 742 3
-339 182 5
-551 38 1
-624 286 5
-320 238 4
-280 323 2
-144 1 4
-280 588 5
-527 433 4
-442 117 3
-222 1239 2
-145 1090 2
-332 550 5
-935 286 5
-747 347 5
-295 809 4
-807 408 3
-378 286 5
-405 638 1
-855 86 2
-2 282 4
-332 449 4
-3 347 5
-483 91 3
-327 421 2
-804 290 4
-378 1425 2
-542 721 2
-869 25 2
-452 502 2
-381 1098 4
-645 195 4
-89 121 5
-786 504 4
-655 1646 3
-479 752 3
-344 647 4
-374 1218 2
-871 17 3
-918 659 4
-115 79 4
-207 79 4
-372 234 5
-314 158 3
-108 405 3
-275 154 2
-768 475 2
-453 566 3
-882 1015 3
-786 173 4
-806 403 4
-262 1014 5
-57 975 3
-253 465 5
-629 196 4
-586 23 2
-337 371 4
-456 963 4
-151 118 3
-788 162 3
-90 1204 4
-420 331 3
-887 1473 1
-707 730 3
-534 150 3
-112 690 4
-425 53 4
-649 1 5
-174 80 1
-694 470 4
-94 227 3
-715 735 4
-5 454 1
-355 1175 5
-500 135 5
-83 28 4
-497 1 4
-655 547 4
-237 179 4
-593 699 4
-387 844 5
-661 515 5
-234 513 5
-295 159 4
-758 896 5
-843 176 4
-13 883 3
-747 1179 1
-625 705 3
-690 158 4
-323 741 3
-663 282 3
-932 890 1
-43 473 3
-660 315 4
-296 948 1
-110 540 3
-330 148 4
-648 1060 2
-270 244 3
-504 1090 4
-11 54 3
-280 409 3
-580 3 5
-405 941 1
-796 821 4
-94 750 4
-189 100 4
-154 475 4
-102 768 2
-530 333 3
-500 396 3
-392 255 3
-829 294 2
-31 321 4
-393 90 2
-234 87 3
-416 191 5
-293 252 2
-870 218 4
-335 322 4
-433 682 2
-624 298 4
-393 1 3
-707 799 4
-31 498 4
-489 266 5
-126 681 5
-130 827 4
-201 767 4
-551 636 5
-456 182 3
-642 257 5
-116 346 4
-221 215 4
-637 685 3
-757 100 3
-62 82 4
-222 271 4
-416 479 5
-466 405 3
-455 2 4
-151 378 4
-276 39 3
-16 56 5
-624 411 4
-215 168 5
-23 211 4
-894 312 3
-128 603 5
-776 670 3
-932 490 4
-196 242 3
-308 479 5
-586 427 3
-55 121 3
-928 134 5
-279 1428 3
-392 134 5
-59 625 3
-66 825 3
-889 652 5
-348 240 3
-255 245 1
-407 144 3
-594 292 3
-715 88 3
-213 7 4
-99 328 4
-116 328 3
-545 95 4
-380 416 2
-314 476 5
-103 126 5
-291 27 3
-940 205 3
-339 525 5
-927 174 3
-489 319 3
-230 144 3
-429 357 5
-721 22 5
-632 845 4
-109 636 5
-527 657 4
-94 368 2
-654 284 4
-420 275 5
-886 410 4
-640 169 5
-889 29 3
-39 306 3
-805 1002 1
-548 284 3
-145 230 5
-798 95 5
-395 739 3
-664 494 5
-417 993 3
-699 211 1
-2 272 5
-291 1016 4
-551 229 5
-804 702 2
-533 98 4
-81 237 4
-52 151 5
-21 358 3
-653 56 5
-301 1283 4
-417 501 3
-758 427 4
-859 1061 4
-431 286 4
-776 440 2
-3 325 1
-343 1067 3
-110 1247 2
-62 306 4
-811 289 2
-402 7 4
-660 658 1
-468 82 5
-379 651 4
-727 596 4
-437 683 2
-833 191 4
-131 287 4
-624 824 2
-287 11 5
-5 451 1
-721 874 3
-18 639 4
-145 327 5
-42 151 4
-405 75 2
-127 288 5
-399 96 3
-524 480 4
-903 147 3
-903 11 2
-648 527 4
-463 1007 3
-551 143 4
-621 270 4
-878 662 1
-488 100 2
-363 854 1
-28 96 5
-468 462 4
-709 172 5
-204 300 3
-280 70 4
-648 205 3
-643 956 4
-42 387 3
-109 715 2
-272 96 5
-509 258 4
-660 569 2
-748 208 4
-291 53 5
-543 190 5
-455 24 3
-901 435 5
-6 258 2
-458 83 4
-276 915 4
-299 496 3
-363 328 3
-655 1319 3
-561 735 3
-364 268 3
-14 588 4
-796 389 4
-79 902 3
-215 226 4
-519 333 3
-932 1121 5
-903 23 5
-588 1311 1
-401 484 3
-213 841 4
-263 245 4
-889 771 2
-491 129 4
-855 855 4
-709 859 3
-332 257 4
-59 1110 4
-128 50 4
-545 96 5
-284 748 3
-321 479 4
-10 195 4
-405 391 1
-659 345 4
-334 537 4
-916 557 4
-795 68 3
-301 284 4
-802 56 3
-382 14 3
-21 928 3
-314 405 4
-157 255 3
-374 475 1
-653 38 3
-621 894 1
-64 97 3
-655 1147 3
-303 22 5
-919 471 3
-216 1101 4
-96 474 4
-623 183 3
-587 261 3
-391 490 4
-532 143 4
-178 82 5
-176 1097 4
-843 378 2
-840 1451 5
-175 186 4
-622 1303 2
-299 14 4
-798 158 2
-896 942 4
-518 696 5
-695 300 1
-320 358 4
-751 751 4
-804 291 4
-279 482 4
-279 283 3
-574 327 3
-561 48 4
-851 310 5
-908 204 4
-378 203 4
-170 299 3
-537 392 2
-897 470 4
-331 180 5
-642 560 4
-371 1 4
-774 64 3
-833 664 3
-97 484 3
-450 205 4
-177 642 4
-551 1136 5
-497 420 3
-783 299 5
-474 664 4
-747 483 5
-889 509 2
-655 778 2
-244 1047 2
-709 68 5
-894 288 3
-943 195 4
-144 282 4
-527 507 5
-898 310 4
-851 875 5
-532 12 5
-598 690 3
-847 98 4
-541 1078 4
-433 173 4
-655 746 3
-795 771 3
-322 591 3
-848 480 5
-158 186 3
-833 1231 4
-445 1534 1
-127 222 5
-363 940 2
-95 657 5
-263 176 5
-890 97 4
-21 769 1
-429 87 3
-850 69 5
-297 659 4
-160 192 5
-393 378 4
-550 15 5
-144 405 4
-223 411 1
-312 921 5
-234 289 4
-606 138 3
-932 474 5
-661 165 5
-60 172 4
-454 431 3
-295 357 4
-667 283 4
-889 700 3
-854 322 1
-825 288 1
-564 312 3
-286 372 4
-461 347 4
-339 200 5
-90 276 4
-872 328 4
-653 1101 2
-815 732 5
-757 679 4
-429 298 5
-38 234 5
-507 892 5
-56 161 4
-374 1047 3
-343 4 5
-683 56 5
-809 340 4
-748 650 1
-345 196 5
-393 731 3
-798 756 3
-939 9 5
-372 326 4
-577 194 4
-279 67 4
-727 451 5
-326 88 2
-314 379 3
-576 276 3
-321 287 3
-301 77 3
-22 403 5
-244 743 5
-382 134 3
-474 461 5
-663 332 4
-784 268 3
-828 1005 3
-677 117 4
-357 125 5
-154 651 4
-199 1326 3
-391 288 3
-38 384 5
-884 213 4
-209 16 4
-659 79 4
-62 1091 3
-268 13 3
-263 87 4
-795 710 3
-826 373 3
-780 660 3
-478 7 1
-198 234 3
-599 866 2
-169 604 4
-417 474 4
-194 51 4
-842 754 1
-647 79 4
-932 1204 5
-535 628 4
-881 705 1
-392 169 4
-83 465 4
-727 147 3
-417 413 3
-59 126 5
-402 127 5
-661 169 5
-222 68 4
-896 231 1
-207 224 3
-293 678 2
-405 1166 1
-666 168 4
-892 58 4
-496 652 5
-154 474 5
-567 646 5
-43 756 3
-758 14 5
-846 693 5
-606 204 4
-446 879 3
-682 62 3
-82 481 5
-561 58 3
-666 742 3
-655 750 2
-534 1028 5
-89 150 5
-778 157 3
-450 204 4
-537 958 2
-894 751 3
-406 1118 3
-886 10 3
-363 47 5
-880 62 3
-487 48 2
-788 498 5
-347 246 4
-763 47 3
-903 192 5
-255 405 4
-896 48 4
-474 380 4
-479 688 1
-56 930 3
-883 279 3
-13 803 3
-299 640 3
-748 514 4
-306 285 4
-503 580 3
-928 176 3
-297 751 4
-357 476 3
-567 648 4
-178 204 4
-682 583 2
-509 266 1
-131 286 5
-405 772 1
-749 257 3
-648 33 1
-758 1034 4
-194 570 3
-817 546 4
-764 1012 4
-401 588 2
-201 332 2
-891 1028 3
-654 98 5
-455 724 3
-823 401 4
-357 412 2
-749 712 3
-780 467 3
-288 427 5
-639 488 4
-276 769 1
-922 739 3
-157 250 1
-71 181 3
-2 311 5
-293 181 3
-276 406 2
-459 105 4
-197 228 4
-423 328 1
-911 427 3
-225 286 4
-331 933 3
-280 739 3
-87 233 4
-536 385 4
-718 240 1
-629 64 5
-264 269 5
-280 1112 4
-374 121 4
-270 159 4
-896 1011 2
-776 436 4
-936 818 4
-519 878 5
-794 514 5
-311 1222 3
-645 640 4
-822 91 3
-436 925 4
-117 977 3
-847 108 2
-905 301 4
-807 82 4
-456 1240 3
-655 1010 3
-5 382 5
-896 173 5
-313 133 5
-100 1233 3
-222 552 2
-829 222 4
-301 121 4
-675 321 2
-572 1010 2
-10 705 4
-833 379 2
-87 435 5
-57 245 4
-481 648 5
-932 615 5
-729 333 4
-417 173 5
-542 318 4
-588 729 3
-563 412 2
-693 1135 3
-880 144 5
-465 584 3
-735 475 4
-880 137 4
-543 810 3
-264 659 5
-798 929 3
-592 237 4
-843 1157 3
-466 898 1
-361 657 5
-267 1336 1
-663 181 4
-296 96 5
-798 825 3
-749 621 3
-751 239 4
-117 597 4
-26 476 3
-807 96 3
-21 330 4
-303 62 2
-852 358 3
-690 85 1
-313 523 5
-617 192 5
-322 194 5
-746 22 4
-698 988 1
-293 155 2
-453 963 4
-222 457 1
-833 5 1
-373 175 3
-846 482 5
-380 180 2
-653 356 1
-502 879 3
-645 664 4
-711 1014 4
-90 732 5
-334 427 4
-432 294 4
-264 240 4
-451 266 2
-279 1206 5
-413 515 5
-709 214 1
-478 182 5
-593 158 3
-62 97 2
-405 1274 1
-561 458 4
-19 692 3
-807 404 3
-589 340 1
-679 109 3
-655 435 2
-543 1099 4
-633 234 4
-344 79 4
-779 181 5
-497 451 2
-638 511 3
-121 250 2
-572 319 4
-627 792 4
-115 1008 5
-527 156 3
-721 333 3
-373 172 5
-250 200 5
-816 343 4
-224 325 1
-766 230 3
-69 282 3
-653 127 5
-907 79 5
-89 936 5
-532 795 2
-739 97 5
-94 722 2
-292 20 2
-843 195 4
-605 269 4
-831 150 3
-189 639 4
-851 258 4
-632 191 5
-394 679 3
-405 1249 1
-846 55 5
-896 710 4
-543 324 3
-591 792 4
-814 288 4
-698 487 2
-387 11 3
-442 68 3
-280 585 3
-198 27 2
-160 825 2
-910 182 4
-232 638 5
-218 269 4
-708 274 4
-303 11 4
-405 1305 1
-838 60 4
-851 347 5
-715 64 5
-877 269 4
-777 527 4
-870 28 4
-938 1012 5
-648 748 3
-379 511 4
-932 470 3
-96 238 4
-294 322 1
-757 248 4
-942 174 5
-83 1101 2
-789 508 4
-272 234 4
-834 276 5
-883 55 4
-839 866 2
-110 232 3
-253 15 4
-655 1161 3
-432 1012 5
-121 118 2
-194 1211 2
-151 748 2
-305 153 3
-425 1595 2
-87 111 4
-276 181 5
-327 684 4
-356 286 3
-308 605 4
-222 783 2
-346 742 4
-715 1222 2
-394 227 4
-96 195 5
-332 831 3
-299 286 4
-303 106 2
-326 219 2
-152 410 4
-665 194 3
-683 268 4
-575 427 4
-495 174 5
-881 70 2
-466 55 4
-318 517 3
-880 1023 2
-279 779 3
-749 378 5
-896 1284 2
-782 298 4
-18 392 3
-13 910 2
-907 763 5
-880 302 5
-648 510 5
-185 528 4
-733 713 4
-840 168 5
-56 238 5
-136 223 4
-207 1049 3
-266 14 4
-221 1017 4
-537 299 2
-256 988 4
-344 244 3
-158 125 3
-178 118 4
-182 203 3
-532 990 3
-682 124 2
-314 468 4
-11 28 5
-110 385 3
-894 689 3
-488 662 4
-92 145 2
-682 191 3
-94 190 5
-505 177 3
-601 378 2
-667 285 5
-466 17 5
-551 317 5
-907 291 5
-763 732 3
-20 496 5
-422 452 3
-611 873 3
-330 105 4
-10 294 3
-429 562 2
-405 577 3
-222 770 3
-478 218 3
-886 425 4
-405 707 1
-15 473 1
-291 218 4
-145 825 4
-351 245 3
-178 1051 3
-158 651 5
-176 7 5
-584 450 2
-207 28 4
-94 155 2
-579 1074 3
-505 1 3
-234 317 2
-18 497 4
-234 378 4
-588 566 2
-908 124 3
-29 874 4
-766 179 4
-521 833 2
-773 531 5
-758 300 2
-880 381 4
-561 201 3
-479 609 5
-781 180 4
-303 919 4
-271 707 4
-401 197 4
-642 139 1
-430 181 4
-348 121 5
-339 423 3
-659 517 5
-846 101 4
-536 188 3
-653 144 3
-144 831 3
-438 284 2
-435 245 2
-469 923 5
-537 848 3
-7 683 4
-128 458 4
-468 174 5
-292 484 5
-547 328 4
-92 190 4
-181 455 1
-85 52 3
-504 441 4
-886 393 3
-889 385 3
-207 660 4
-805 176 4
-535 469 3
-207 58 3
-908 123 3
-450 559 3
-452 22 5
-751 237 2
-291 1078 4
-719 742 4
-830 288 1
-933 9 3
-648 298 2
-848 173 5
-640 81 5
-436 425 4
-1 36 2
-719 294 2
-264 123 4
-599 1315 4
-540 274 4
-151 516 5
-758 1098 5
-59 18 4
-347 288 5
-152 204 4
-746 385 5
-747 625 3
-659 181 3
-846 640 1
-270 283 5
-407 56 5
-424 15 4
-639 269 3
-223 258 1
-455 518 4
-456 1081 4
-618 697 3
-281 289 3
-397 194 3
-608 508 4
-437 301 3
-1 64 5
-749 433 3
-437 501 4
-98 938 3
-711 272 5
-756 398 3
-871 1386 3
-370 153 2
-690 25 3
-77 636 2
-774 232 2
-327 455 2
-545 98 5
-279 301 4
-26 257 3
-116 748 2
-669 205 4
-279 1011 3
-587 331 3
-338 168 3
-21 982 1
-463 347 1
-129 269 4
-308 684 3
-825 250 5
-393 181 4
-130 210 5
-804 208 5
-606 845 4
-401 654 3
-384 329 3
-350 50 5
-654 195 4
-504 729 5
-458 496 3
-716 367 4
-758 147 4
-12 82 4
-36 883 5
-239 632 5
-393 685 3
-409 204 5
-254 386 2
-847 1204 3
-885 402 3
-805 659 3
-629 132 5
-790 51 3
-623 288 1
-707 317 3
-766 212 5
-707 506 2
-63 591 3
-705 83 4
-861 547 4
-22 187 5
-52 15 5
-624 597 3
-197 187 5
-537 300 1
-18 705 3
-474 205 5
-59 483 5
-275 448 3
-268 559 2
-778 11 5
-501 410 4
-472 49 5
-847 141 3
-848 82 5
-425 455 2
-388 678 4
-523 56 3
-924 174 5
-666 202 5
-15 455 1
-529 324 2
-338 180 4
-405 302 4
-653 517 1
-786 208 5
-6 21 3
-618 68 3
-707 167 2
-76 203 4
-656 301 3
-349 713 3
-907 312 5
-328 451 4
-87 121 5
-896 423 3
-766 605 3
-506 607 4
-664 516 5
-393 111 3
-222 1029 1
-496 288 2
-342 663 4
-487 596 5
-221 230 3
-391 282 4
-380 174 4
-393 391 3
-536 603 4
-634 690 3
-913 4 4
-474 653 4
-709 564 1
-313 499 3
-43 409 3
-109 12 4
-489 351 5
-880 795 2
-900 137 3
-897 528 3
-49 657 5
-864 768 3
-584 40 4
-749 496 5
-870 202 3
-760 928 1
-492 654 4
-447 98 4
-49 531 3
-756 289 4
-655 311 3
-85 664 4
-85 1167 3
-495 1119 4
-291 597 3
-838 713 4
-929 419 4
-504 234 3
-655 582 2
-474 611 4
-551 13 1
-814 7 4
-682 657 4
-362 322 3
-215 174 4
-288 182 4
-495 218 4
-721 423 5
-470 813 3
-110 783 3
-405 197 4
-870 51 2
-587 323 4
-622 386 3
-735 258 4
-214 568 4
-647 554 4
-918 1639 5
-243 191 5
-387 845 4
-896 70 4
-119 209 4
-314 144 3
-863 315 5
-291 284 4
-85 300 3
-871 342 4
-533 66 4
-489 349 4
-280 631 5
-342 223 4
-207 281 3
-170 300 5
-666 499 4
-467 276 5
-731 207 4
-436 739 4
-128 705 3
-287 546 4
-291 977 2
-468 273 2
-6 59 5
-6 518 3
-846 173 4
-325 109 2
-870 1042 2
-254 423 5
-360 238 4
-280 1473 3
-425 269 4
-306 756 3
-346 161 3
-786 97 4
-666 483 5
-316 14 4
-393 31 4
-864 399 4
-279 40 4
-782 1256 2
-682 17 3
-588 367 5
-788 234 3
-640 202 5
-864 466 4
-13 526 3
-417 831 2
-887 633 5
-378 433 4
-302 307 4
-458 357 3
-807 421 3
-486 1226 4
-889 470 4
-669 114 5
-21 396 2
-396 1215 2
-894 1005 5
-11 399 3
-597 289 5
-7 455 4
-840 650 4
-752 346 4
-7 672 1
-926 245 3
-650 472 3
-59 871 2
-103 300 3
-787 906 1
-682 876 3
-632 164 4
-854 522 2
-815 647 5
-308 699 4
-459 832 3
-404 286 1
-186 243 2
-559 385 4
-419 173 5
-848 419 5
-94 118 3
-625 393 4
-493 7 3
-416 153 4
-312 463 5
-162 943 4
-411 229 3
-13 485 1
-747 276 5
-305 268 3
-766 91 5
-435 265 3
-798 164 4
-286 746 4
-194 67 1
-764 100 4
-393 64 4
-378 588 5
-281 332 4
-745 507 1
-716 432 5
-833 55 3
-655 707 3
-377 678 2
-933 28 4
-211 127 4
-193 580 4
-401 473 1
-651 127 4
-177 1039 3
-514 1039 5
-288 69 5
-118 672 4
-64 69 4
-854 476 3
-648 410 2
-666 514 4
-7 502 5
-222 508 3
-770 475 5
-746 720 3
-312 641 5
-299 1018 3
-634 288 3
-373 190 5
-932 157 4
-790 208 3
-717 324 3
-606 1149 4
-429 234 4
-1 241 4
-897 96 5
-916 713 3
-13 447 2
-610 71 4
-493 61 4
-810 879 5
-189 1315 3
-230 161 5
-919 275 5
-264 168 5
-537 269 3
-397 853 4
-137 15 4
-222 781 3
-748 50 5
-532 1217 4
-484 746 4
-411 196 4
-82 483 5
-288 214 2
-409 481 3
-609 285 5
-117 132 4
-496 1229 1
-778 1035 1
-407 89 4
-326 385 3
-545 210 5
-653 11 2
-269 739 1
-804 82 5
-439 307 3
-731 136 4
-409 1541 4
-279 753 2
-295 265 4
-303 55 4
-505 181 3
-616 750 5
-602 343 2
-848 1021 5
-758 100 5
-847 289 5
-479 647 5
-916 2 3
-21 103 1
-151 190 4
-898 343 3
-807 930 5
-798 1164 3
-291 627 4
-503 387 4
-13 678 3
-385 487 4
-85 499 4
-454 228 3
-152 111 5
-570 879 2
-11 370 3
-92 149 3
-97 168 4
-806 14 3
-878 172 4
-221 623 3
-557 865 3
-25 82 4
-567 611 4
-269 134 4
-871 751 4
-500 164 4
-290 205 3
-303 1014 3
-516 902 5
-450 173 5
-553 218 4
-271 1117 3
-379 144 5
-670 195 4
-823 651 5
-358 584 4
-886 273 2
-892 1091 2
-629 699 3
-524 501 2
-907 118 4
-280 540 3
-507 316 5
-234 552 2
-380 286 5
-506 202 5
-71 65 5
-541 82 3
-280 419 3
-405 347 4
-711 1053 4
-389 493 5
-690 94 4
-916 652 4
-790 1119 4
-8 172 5
-301 474 4
-843 205 4
-500 77 3
-749 280 4
-804 647 5
-274 742 4
-733 276 5
-932 82 3
-275 222 4
-262 411 2
-342 92 4
-537 1166 2
-298 427 5
-90 302 5
-862 45 4
-847 95 4
-389 383 2
-206 882 1
-18 414 4
-460 306 4
-868 69 2
-425 233 2
-501 544 4
-908 47 3
-871 402 3
-99 273 5
-561 238 4
-750 300 3
-207 211 5
-527 12 4
-14 111 3
-711 65 4
-207 385 3
-42 276 1
-230 174 5
-428 289 4
-452 521 3
-109 252 5
-49 270 2
-663 180 4
-334 310 3
-109 1210 3
-569 756 3
-22 29 1
-389 1530 2
-82 588 5
-644 326 5
-144 815 1
-857 19 4
-795 381 2
-23 294 1
-903 1067 2
-453 100 5
-486 147 2
-21 298 5
-95 33 3
-717 117 4
-856 313 5
-454 1063 4
-116 298 3
-698 143 3
-653 328 4
-130 89 4
-705 393 4
-6 367 2
-758 488 3
-704 604 5
-328 172 4
-717 50 4
-234 125 3
-34 690 4
-695 305 3
-564 302 3
-312 863 5
-62 921 2
-665 69 5
-896 238 3
-200 717 4
-307 395 3
-292 151 5
-89 197 5
-537 42 3
-584 249 4
-806 3 2
-366 443 5
-344 269 4
-380 172 3
-276 523 4
-280 145 3
-932 414 4
-269 22 1
-883 172 4
-472 101 5
-336 949 4
-404 301 3
-727 751 3
-778 441 3
-13 323 3
-632 470 4
-169 482 3
-234 1078 2
-502 323 4
-509 288 5
-219 855 5
-717 1282 4
-840 238 5
-417 214 5
-286 1316 5
-45 764 4
-467 109 5
-207 238 2
-666 1451 3
-846 174 5
-548 1405 3
-796 132 4
-262 1095 2
-301 387 3
-158 190 5
-543 919 2
-795 234 4
-676 250 4
-494 121 4
-18 762 3
-145 5 3
-95 227 2
-853 330 1
-785 301 4
-365 269 4
-339 154 4
-269 8 2
-269 367 3
-145 352 4
-829 705 4
-17 475 4
-890 663 4
-814 358 2
-425 853 4
-14 173 4
-508 451 3
-630 257 3
-695 323 2
-450 785 3
-26 9 4
-943 415 1
-442 350 2
-297 28 4
-817 7 4
-284 304 4
-499 482 2
-743 9 5
-128 468 1
-795 62 4
-130 731 3
-524 64 2
-227 322 3
-624 886 4
-21 288 3
-7 416 5
-393 394 5
-642 64 5
-85 269 3
-862 214 3
-13 823 5
-119 117 5
-834 181 5
-64 476 1
-269 475 5
-253 699 4
-654 288 3
-57 225 3
-782 1600 3
-936 333 3
-499 257 5
-480 89 4
-7 641 5
-681 682 1
-807 385 4
-27 50 3
-248 194 4
-846 44 1
-354 382 5
-554 596 3
-627 849 4
-387 23 2
-108 100 4
-842 1395 4
-538 566 3
-198 690 3
-854 328 1
-60 517 4
-293 729 2
-587 334 3
-314 762 4
-788 744 4
-798 1076 3
-201 233 4
-190 237 5
-663 23 4
-267 739 4
-551 756 1
-528 657 5
-123 523 3
-47 340 5
-66 294 4
-553 520 5
-892 367 4
-527 124 4
-258 893 1
-650 95 3
-846 39 3
-936 678 3
-128 1039 4
-256 974 3
-311 275 4
-760 120 1
-455 1174 3
-591 25 4
-60 153 3
-393 313 4
-889 646 3
-232 750 3
-18 285 5
-527 143 2
-409 657 3
-471 50 3
-23 747 3
-669 216 3
-889 12 5
-840 175 4
-166 322 5
-387 144 3
-693 282 4
-344 172 4
-745 492 5
-776 637 3
-437 15 4
-860 712 3
-345 282 3
-797 336 2
-301 423 1
-796 487 5
-271 200 5
-437 523 3
-91 132 3
-664 692 3
-916 679 3
-158 562 4
-234 174 3
-554 288 3
-533 402 4
-745 100 5
-406 22 3
-515 750 2
-327 1069 4
-919 1284 3
-504 731 3
-758 238 5
-878 949 3
-796 1299 2
-595 1010 4
-350 604 5
-509 245 2
-216 215 5
-782 299 3
-553 23 5
-474 518 4
-822 902 4
-798 88 4
-864 660 4
-622 298 4
-178 245 3
-428 272 5
-537 191 4
-747 580 5
-156 346 3
-25 204 5
-437 381 5
-270 1074 5
-416 734 3
-635 255 4
-880 294 4
-94 313 4
-204 1296 5
-268 1090 2
-749 132 4
-295 241 5
-328 215 3
-548 255 4
-840 644 4
-844 1474 4
-297 53 3
-399 78 3
-883 100 4
-303 582 4
-135 173 4
-286 154 4
-295 1133 4
-451 263 2
-484 4 4
-576 259 2
-510 289 2
-878 57 4
-870 289 2
-110 651 4
-505 496 5
-234 66 3
-332 118 5
-592 298 5
-486 1202 4
-459 3 2
-11 259 3
-429 134 5
-758 362 5
-486 926 2
-864 178 4
-248 294 3
-372 292 5
-666 646 3
-429 433 3
-411 168 5
-378 1531 4
-808 313 5
-184 1008 4
-409 855 4
-515 307 4
-365 315 4
-921 185 3
-409 207 3
-328 751 3
-190 258 3
-916 451 3
-268 120 2
-622 419 4
-403 111 4
-325 32 3
-44 21 2
-276 391 2
-807 612 5
-715 174 4
-606 250 4
-904 280 5
-204 286 3
-98 88 3
-561 151 2
-860 269 2
-567 9 4
-634 272 5
-555 762 4
-658 171 4
-626 294 3
-23 90 2
-627 328 4
-479 183 5
-412 208 4
-889 209 2
-221 809 3
-381 1532 2
-653 510 2
-102 522 3
-577 226 4
-334 276 4
-863 1062 4
-16 404 5
-201 847 2
-913 235 1
-842 752 4
-91 510 3
-214 69 2
-151 230 3
-591 13 4
-660 393 2
-201 1103 3
-409 1393 1
-349 288 3
-6 248 3
-833 240 4
-380 423 3
-534 1054 5
-889 1194 4
-110 204 3
-351 341 4
-893 11 4
-195 1418 4
-178 679 4
-459 866 5
-416 248 5
-7 197 4
-222 64 5
-465 87 4
-727 187 5
-390 742 4
-733 676 4
-342 866 1
-720 995 4
-711 729 3
-552 1014 4
-299 270 4
-883 709 5
-207 1331 3
-868 447 2
-384 343 3
-13 890 1
-322 489 3
-530 483 3
-405 1246 1
-402 276 5
-533 243 3
-790 98 5
-276 1220 4
-286 655 3
-53 257 4
-865 685 3
-433 137 5
-648 98 4
-589 751 4
-682 1 4
-472 176 5
-886 26 4
-524 451 3
-877 702 4
-437 276 5
-223 846 2
-504 307 4
-458 631 4
-272 12 5
-578 298 4
-777 212 5
-305 13 3
-54 148 3
-277 628 4
-145 750 4
-870 192 5
-607 485 3
-618 204 3
-632 131 4
-144 126 4
-332 693 5
-605 133 5
-13 21 3
-429 1301 4
-592 286 5
-709 672 2
-60 82 3
-911 435 5
-195 313 5
-894 509 4
-655 660 2
-804 663 5
-666 523 4
-197 530 3
-650 484 5
-634 676 4
-410 303 3
-574 332 3
-747 32 5
-466 210 4
-18 660 5
-28 447 3
-343 194 5
-83 319 1
-761 1152 2
-900 471 2
-156 12 3
-551 761 1
-606 404 4
-201 65 4
-416 972 4
-110 79 4
-405 415 2
-834 117 4
-804 199 5
-559 197 4
-299 166 4
-680 151 5
-42 284 3
-486 242 4
-913 99 4
-798 1063 3
-889 121 4
-846 566 5
-466 684 4
-580 148 4
-323 9 4
-113 277 3
-347 210 4
-648 550 4
-33 880 3
-158 770 5
-592 1275 3
-145 338 3
-919 558 5
-545 78 2
-417 147 4
-496 421 3
-121 257 5
-416 126 5
-11 549 4
-43 258 5
-46 245 3
-423 355 3
-9 276 4
-664 724 3
-552 225 3
-776 661 5
-597 328 4
-394 1484 4
-653 502 2
-689 118 4
-741 566 4
-766 62 3
-429 684 4
-481 98 4
-325 127 5
-890 210 4
-293 317 4
-709 65 2
-345 200 4
-342 3 2
-102 546 3
-2 276 4
-6 483 5
-339 655 4
-13 862 3
-355 1233 4
-758 139 4
-43 1052 1
-804 1079 4
-659 174 4
-345 470 4
-757 151 4
-327 396 3
-399 318 5
-525 475 3
-455 193 4
-452 288 2
-790 708 3
-123 504 5
-766 214 2
-212 268 5
-126 272 3
-588 40 4
-815 434 3
-338 134 5
-871 359 3
-864 96 5
-305 1485 3
-276 1483 3
-721 197 4
-551 51 5
-372 1212 4
-449 282 3
-847 448 4
-938 472 4
-31 303 3
-238 111 4
-249 117 4
-528 238 3
-508 47 3
-712 783 3
-666 692 3
-268 241 3
-104 544 3
-370 135 4
-843 79 2
-393 191 3
-885 100 3
-288 1065 4
-116 896 2
-222 431 4
-299 498 4
-754 127 4
-141 258 5
-853 1025 4
-932 431 3
-7 294 1
-851 1245 4
-877 382 3
-58 13 3
-13 663 5
-90 23 5
-72 479 4
-449 251 3
-843 625 2
-601 284 4
-650 565 3
-458 823 3
-249 317 5
-524 275 3
-604 637 4
-770 250 5
-529 1038 4
-237 83 4
-815 945 4
-320 458 4
-497 746 5
-271 13 4
-427 334 5
-17 508 3
-896 732 4
-201 303 2
-798 828 4
-222 1041 3
-159 323 4
-550 313 5
-449 971 4
-430 674 4
-5 90 3
-489 245 3
-207 64 5
-363 232 2
-207 181 3
-320 739 4
-886 475 5
-15 546 2
-650 601 3
-347 235 2
-405 450 1
-741 204 4
-344 1050 3
-323 121 3
-119 472 4
-409 1070 4
-474 519 4
-397 338 4
-819 286 5
-94 282 3
-90 499 5
-85 340 3
-454 181 3
-536 205 5
-393 586 3
-576 100 4
-776 22 5
-582 1 4
-847 168 4
-871 275 3
-887 128 5
-654 332 4
-90 179 5
-311 498 4
-189 241 3
-354 629 3
-334 608 4
-389 182 5
-543 82 4
-804 181 5
-85 210 3
-901 1035 4
-790 1016 2
-313 73 5
-727 91 4
-790 949 4
-280 381 3
-864 892 3
-216 249 3
-894 316 4
-630 31 2
-437 215 3
-608 489 5
-653 474 4
-639 512 2
-551 1047 4
-313 661 4
-234 530 4
-374 1134 4
-162 508 5
-303 99 4
-787 328 3
-686 435 5
-10 333 4
-177 475 4
-815 659 5
-172 642 4
-296 222 5
-798 722 3
-414 260 3
-330 427 5
-747 178 5
-916 1046 2
-184 604 4
-177 963 4
-741 172 5
-757 328 3
-879 292 4
-922 271 3
-456 324 4
-937 988 2
-711 423 3
-92 123 2
-210 204 5
-378 1053 3
-872 975 4
-606 79 3
-25 968 4
-13 230 3
-503 452 1
-399 549 4
-177 628 2
-331 514 3
-592 1377 3
-671 17 4
-694 200 4
-194 770 4
-804 134 4
-323 268 4
-795 477 3
-442 742 3
-854 537 3
-620 895 3
-538 208 3
-417 1119 3
-673 323 2
-305 663 3
-871 909 3
-392 209 5
-109 1012 4
-3 324 2
-918 133 1
-711 98 5
-533 371 3
-854 471 2
-796 449 4
-12 242 5
-940 751 3
-445 9 2
-665 419 4
-339 205 5
-128 82 5
-746 568 4
-392 249 1
-305 302 4
-436 742 5
-828 510 3
-761 1014 1
-711 317 4
-655 86 4
-327 175 2
-399 268 3
-256 31 5
-796 873 3
-519 895 4
-487 215 4
-286 81 3
-930 171 1
-698 22 1
-599 763 5
-262 582 4
-82 211 4
-405 204 5
-502 288 5
-880 173 3
-75 408 4
-666 707 5
-653 136 1
-280 476 5
-532 72 3
-711 218 4
-521 568 3
-262 443 3
-184 949 3
-472 172 5
-91 322 4
-580 249 5
-871 1176 3
-712 794 4
-523 863 4
-234 484 5
-653 511 4
-95 640 3
-60 507 4
-291 4 4
-248 176 5
-435 983 2
-256 4 5
-378 174 4
-642 734 3
-828 86 3
-918 709 4
-727 576 4
-151 385 3
-486 281 3
-296 498 5
-320 774 4
-296 315 5
-763 972 3
-303 544 4
-28 227 4
-382 122 3
-394 928 4
-206 272 5
-901 405 4
-776 607 4
-92 678 2
-313 428 3
-524 1041 2
-480 127 3
-194 73 3
-896 282 2
-223 282 4
-320 946 5
-85 25 2
-546 271 5
-457 472 4
-87 996 3
-478 232 2
-458 69 2
-456 546 4
-271 435 4
-116 7 2
-531 748 4
-117 237 4
-121 98 5
-406 425 3
-552 410 3
-264 202 5
-262 786 3
-109 871 2
-825 866 4
-222 432 3
-327 183 3
-828 985 3
-929 480 3
-425 355 3
-363 705 2
-112 354 3
-394 88 3
-455 255 2
-14 302 5
-749 182 3
-747 235 5
-498 203 5
-943 194 5
-338 197 5
-553 483 5
-768 332 4
-883 421 5
-271 132 5
-516 250 4
-892 12 5
-715 685 3
-510 330 2
-13 118 4
-222 470 3
-393 139 4
-866 302 2
-352 79 4
-452 531 4
-250 458 5
-405 1549 1
-537 901 1
-758 1143 5
-644 1025 4
-505 193 3
-102 501 2
-457 31 4
-28 28 4
-850 435 4
-543 212 4
-43 284 5
-339 484 5
-708 127 3
-733 285 4
-171 906 3
-717 975 2
-495 56 5
-15 620 4
-332 356 3
-635 327 5
-500 739 2
-569 471 3
-13 840 3
-899 403 3
-846 661 4
-621 1013 2
-291 735 4
-479 261 1
-887 228 4
-837 16 2
-151 512 5
-606 959 5
-421 525 4
-542 282 3
-464 1025 2
-899 89 4
-843 214 3
-7 417 3
-145 228 4
-94 203 5
-774 530 5
-210 722 4
-173 687 1
-269 179 4
-22 502 4
-711 472 1
-63 108 2
-671 184 3
-181 1152 2
-682 290 1
-305 664 2
-698 499 3
-589 300 5
-707 294 2
-921 1317 2
-822 588 2
-800 276 3
-932 484 5
-305 686 3
-537 301 2
-339 719 3
-489 872 2
-846 1124 4
-64 662 4
-104 147 3
-442 1074 3
-718 744 3
-168 871 3
-758 234 4
-486 303 4
-59 134 5
-897 472 5
-110 684 4
-264 19 5
-144 1138 4
-807 684 5
-894 591 4
-527 183 5
-38 405 5
-178 751 4
-617 174 1
-894 736 4
-422 559 3
-676 303 4
-229 344 5
-8 510 4
-474 276 5
-172 220 4
-644 294 4
-918 658 3
-836 163 5
-130 729 4
-851 238 5
-380 610 2
-213 778 5
-99 762 2
-620 1035 4
-484 234 4
-838 191 5
-64 82 3
-880 810 3
-194 185 4
-766 50 4
-759 756 4
-6 519 5
-406 921 4
-719 582 3
-537 873 2
-567 176 5
-94 419 3
-936 9 4
-269 525 4
-659 317 4
-299 311 4
-308 485 3
-928 168 5
-343 9 5
-450 1603 3
-280 73 3
-466 11 3
-561 455 3
-27 475 2
-267 240 4
-796 178 3
-437 162 4
-868 199 5
-405 1177 1
-90 837 5
-560 1 4
-371 527 5
-56 385 4
-192 111 2
-532 267 3
-189 484 5
-821 483 5
-209 408 4
-342 7 4
-503 14 3
-916 118 2
-253 483 5
-381 596 3
-504 132 5
-592 272 5
-775 264 4
-76 325 2
-401 515 4
-486 21 3
-235 1119 3
-412 135 4
-190 363 2
-110 22 4
-383 193 4
-642 50 5
-405 1176 3
-880 1013 3
-152 97 5
-141 1 3
-715 204 4
-373 421 4
-846 1411 4
-335 288 4
-290 211 3
-296 100 5
-158 514 3
-60 495 3
-299 1132 1
-932 509 3
-90 703 3
-174 845 5
-104 100 4
-846 638 4
-308 205 3
-304 275 4
-115 302 4
-243 582 5
-58 249 4
-538 381 3
-234 487 3
-655 520 3
-459 1040 2
-271 87 3
-756 1074 4
-18 81 3
-546 98 5
-678 924 2
-194 64 5
-896 820 2
-846 1209 1
-858 678 1
-18 178 3
-326 484 5
-162 28 4
-85 708 4
-621 118 3
-99 471 4
-267 233 4
-387 772 4
-676 1527 1
-214 23 5
-134 313 5
-387 385 3
-919 204 4
-682 243 1
-256 597 4
-435 22 4
-574 1062 5
-880 28 5
-648 713 2
-244 955 4
-860 1602 3
-840 462 3
-823 227 1
-56 164 4
-84 408 5
-875 55 3
-373 402 4
-889 65 4
-561 88 2
-339 42 4
-160 3 3
-688 898 5
-456 658 3
-407 222 4
-932 560 2
-482 876 3
-926 300 3
-582 117 3
-162 117 4
-617 440 4
-387 1187 4
-881 209 3
-438 252 4
-852 25 3
-807 22 5
-314 246 5
-62 462 2
-270 452 4
-301 604 4
-727 259 4
-588 1041 2
-711 22 4
-655 65 2
-184 166 3
-532 931 3
-90 611 5
-782 1016 3
-184 655 3
-381 660 2
-847 444 3
-788 1135 2
-130 291 4
-561 616 3
-262 278 3
-535 188 3
-136 303 4
-524 65 4
-923 340 5
-174 393 4
-94 318 5
-939 409 4
-201 76 4
-756 147 4
-297 86 5
-299 86 4
-442 975 3
-653 1135 2
-758 1159 5
-363 187 2
-59 612 3
-70 214 3
-328 610 3
-315 216 4
-336 15 4
-350 1039 4
-193 1090 2
-883 66 3
-618 576 4
-839 508 3
-280 678 2
-223 259 3
-934 144 4
-92 118 2
-551 185 5
-717 343 4
-892 159 4
-495 11 5
-561 345 4
-83 1047 2
-708 362 1
-222 186 5
-276 959 4
-370 443 5
-119 475 4
-204 874 3
-57 240 2
-293 411 2
-346 50 5
-881 405 4
-505 203 4
-94 257 4
-405 1107 1
-468 724 4
-826 432 3
-312 515 5
-871 515 4
-606 844 4
-659 481 5
-303 287 4
-13 520 4
-622 532 3
-934 657 3
-921 1051 3
-577 739 3
-911 186 5
-379 286 4
-747 156 3
-291 574 1
-223 125 3
-200 465 4
-181 1328 1
-130 268 4
-54 106 3
-250 259 1
-639 483 5
-65 66 3
-321 430 3
-299 749 1
-399 338 1
-452 203 3
-18 195 3
-296 292 5
-288 357 5
-234 429 4
-109 986 2
-263 180 4
-696 124 5
-455 12 3
-798 996 3
-90 750 4
-710 99 4
-757 825 3
-848 496 2
-939 237 5
-724 346 1
-72 1148 4
-796 134 3
-552 934 3
-137 327 4
-788 549 4
-440 300 3
-738 1047 3
-409 499 3
-650 504 3
-314 929 3
-541 477 4
-269 100 5
-276 192 5
-412 487 3
-289 1 3
-293 546 1
-141 120 4
-151 301 4
-450 367 3
-272 474 5
-276 238 5
-430 628 3
-24 237 4
-276 552 3
-23 116 5
-527 210 4
-405 1404 1
-305 79 3
-21 635 4
-405 230 2
-435 673 3
-457 137 5
-932 506 4
-72 124 4
-348 406 4
-13 497 5
-798 72 3
-237 98 4
-6 32 4
-361 234 4
-306 287 4
-373 239 3
-311 204 5
-804 11 4
-327 2 2
-567 487 4
-761 426 1
-592 597 2
-63 924 3
-307 515 4
-721 51 4
-438 300 4
-655 1136 2
-42 1046 3
-279 1492 4
-425 97 2
-499 524 4
-293 36 1
-327 216 3
-5 446 4
-738 225 3
-912 646 3
-537 741 2
-60 60 5
-784 303 4
-472 443 4
-147 319 4
-172 1134 2
-818 751 5
-321 190 4
-870 92 4
-927 69 4
-200 840 4
-70 473 3
-404 22 5
-787 305 3
-683 245 2
-181 1368 1
-537 497 4
-268 116 4
-698 211 2
-128 294 4
-80 699 3
-924 1036 2
-478 71 3
-26 249 2
-291 985 3
-693 159 4
-359 268 4
-493 806 3
-810 304 4
-405 308 1
-38 780 4
-711 755 3
-406 134 5
-165 216 4
-844 151 4
-457 357 5
-374 806 3
-299 742 4
-889 749 2
-301 15 4
-157 407 4
-454 465 3
-236 237 4
-551 3 5
-474 316 5
-627 187 5
-834 326 4
-348 25 4
-924 275 4
-640 186 5
-308 285 5
-184 1014 2
-795 403 3
-688 288 5
-315 657 4
-919 304 4
-791 748 3
-367 234 4
-533 172 4
-918 25 4
-181 276 2
-268 37 3
-548 276 3
-448 321 4
-135 802 2
-716 483 5
-552 284 3
-590 15 3
-59 451 5
-212 382 5
-814 590 2
-648 864 3
-835 294 3
-655 559 2
-432 111 4
-763 498 4
-630 932 2
-634 93 5
-222 218 5
-585 213 5
-275 164 4
-773 919 5
-533 755 3
-554 684 4
-60 1050 3
-56 210 5
-339 286 5
-286 143 4
-686 187 5
-705 210 5
-883 584 3
-553 187 5
-57 294 4
-141 291 5
-160 201 5
-480 294 1
-359 270 4
-608 462 4
-588 110 3
-601 357 4
-378 731 3
-221 576 3
-297 143 5
-592 32 5
-385 346 3
-125 577 2
-864 357 5
-450 620 4
-171 313 4
-416 401 2
-90 610 5
-554 692 4
-505 173 3
-548 344 1
-6 528 4
-943 559 4
-543 513 4
-487 95 4
-924 273 3
-727 363 3
-313 161 4
-562 427 4
-733 107 4
-661 48 4
-23 173 5
-357 24 4
-671 203 3
-797 286 2
-715 158 2
-115 508 5
-294 246 4
-815 143 5
-659 89 4
-702 307 2
-477 722 5
-253 79 5
-60 745 5
-152 411 4
-788 983 3
-716 70 4
-308 770 4
-62 86 2
-472 391 2
-870 1230 2
-672 275 5
-253 747 3
-591 740 4
-279 1093 4
-830 177 4
-363 461 3
-342 461 3
-378 241 4
-807 405 4
-449 60 5
-694 1205 3
-381 1119 4
-43 122 2
-456 204 3
-385 198 3
-457 52 4
-907 1220 5
-643 527 3
-639 311 3
-128 425 5
-503 293 4
-270 684 4
-119 144 4
-694 300 4
-532 483 5
-2 303 4
-533 181 5
-110 566 4
-353 328 2
-350 153 3
-846 42 5
-15 25 3
-448 896 5
-279 173 5
-805 581 2
-327 678 3
-340 215 5
-846 692 3
-593 121 4
-283 412 5
-280 125 2
-650 15 3
-548 405 4
-938 290 3
-586 29 5
-200 758 3
-465 428 3
-332 562 5
-806 1059 3
-488 203 4
-399 127 2
-342 165 3
-474 77 5
-828 921 4
-346 358 4
-63 1007 5
-292 83 5
-320 976 2
-201 70 3
-383 484 4
-943 672 5
-823 459 4
-712 417 4
-624 316 4
-145 134 4
-655 50 4
-329 483 4
-84 100 4
-807 73 3
-290 99 4
-601 260 4
-602 127 5
-663 13 3
-13 665 2
-312 153 2
-197 92 1
-659 616 4
-95 71 5
-848 241 5
-308 79 4
-699 1284 3
-537 1069 2
-239 79 3
-691 170 5
-655 58 3
-606 201 4
-364 990 4
-773 72 3
-406 528 4
-169 443 4
-243 280 1
-148 228 4
-268 374 2
-347 416 3
-833 271 5
-85 58 4
-561 55 4
-496 1614 3
-942 1050 5
-406 50 5
-52 7 5
-853 331 2
-917 282 4
-682 732 3
-311 679 4
-560 1215 2
-351 879 5
-738 117 3
-368 234 3
-627 58 5
-487 941 3
-62 405 3
-680 1012 3
-389 489 4
-230 405 4
-786 322 3
-523 213 5
-776 238 4
-468 39 3
-330 255 4
-230 223 5
-222 806 4
-312 506 4
-437 219 3
-940 692 4
-286 211 4
-871 1072 3
-478 79 4
-377 56 4
-629 880 4
-178 22 5
-453 73 4
-64 300 3
-234 1170 1
-880 273 5
-790 436 4
-6 131 5
-177 59 4
-595 591 4
-194 219 2
-301 50 5
-866 896 2
-276 117 4
-290 118 4
-796 514 3
-760 204 4
-6 165 5
-878 1039 3
-325 542 2
-537 91 2
-500 129 4
-275 257 3
-649 117 5
-541 924 5
-650 71 3
-121 249 1
-512 97 5
-854 176 3
-618 676 2
-500 237 3
-200 826 4
-754 255 3
-682 24 4
-870 253 4
-669 22 3
-22 792 4
-826 1219 4
-655 1160 3
-619 328 1
-255 413 2
-856 294 4
-475 70 4
-890 671 5
-566 98 4
-878 699 1
-756 30 4
-244 676 4
-222 153 4
-358 268 3
-456 979 3
-523 533 4
-383 86 5
-49 299 2
-456 423 3
-158 285 5
-447 144 5
-461 158 2
-264 230 4
-896 24 4
-1 80 4
-551 762 5
-416 174 5
-843 209 3
-416 585 1
-406 235 4
-378 550 2
-486 1082 2
-109 117 5
-650 82 3
-871 56 5
-216 108 4
-407 388 2
-12 97 5
-758 527 5
-579 748 3
-882 473 3
-222 1336 2
-693 568 4
-436 1028 4
-145 98 5
-498 430 4
-693 651 3
-385 715 3
-379 50 4
-354 152 3
-500 611 5
-475 303 1
-749 258 4
-222 216 4
-630 687 3
-894 971 3
-416 95 3
-458 187 5
-362 683 1
-430 1007 3
-903 642 4
-504 717 4
-463 1197 4
-567 430 4
-378 50 4
-378 476 3
-251 275 4
-224 526 4
-716 428 3
-290 204 4
-393 374 3
-312 166 5
-174 369 1
-708 276 2
-87 414 3
-87 427 4
-851 680 3
-806 1074 3
-501 475 5
-466 292 4
-452 641 3
-459 597 3
-200 578 5
-846 58 4
-747 23 5
-521 679 3
-90 131 5
-642 195 3
-342 57 3
-774 712 1
-396 595 3
-703 100 4
-501 276 4
-139 307 4
-802 769 5
-194 744 3
-655 159 3
-862 176 5
-488 434 4
-380 570 3
-552 300 4
-693 333 3
-919 794 4
-751 380 3
-493 881 1
-826 399 4
-299 1020 4
-326 519 5
-377 443 4
-481 190 5
-650 654 3
-716 723 4
-560 756 2
-47 268 4
-207 754 4
-336 1218 3
-363 350 1
-715 627 3
-537 469 3
-907 699 5
-868 59 4
-292 789 4
-178 651 4
-632 1 3
-347 148 3
-537 312 3
-467 7 5
-670 657 5
-102 550 2
-870 66 4
-280 496 5
-606 596 4
-936 1079 1
-764 371 3
-593 723 4
-280 31 4
-353 343 2
-288 15 4
-11 237 4
-363 746 4
-62 190 5
-478 151 5
-409 631 3
-13 694 4
-878 174 3
-435 148 3
-256 100 4
-402 471 4
-632 98 4
-877 228 4
-250 596 5
-94 652 4
-435 174 5
-351 898 5
-313 582 2
-889 405 2
-219 631 5
-561 19 3
-307 431 4
-128 186 5
-345 124 5
-142 189 4
-323 151 4
-70 527 4
-391 9 5
-476 692 3
-102 72 3
-181 3 2
-345 739 4
-92 455 2
-881 654 4
-450 366 3
-648 924 1
-870 487 4
-900 508 3
-87 566 5
-666 566 3
-758 91 4
-541 62 4
-842 269 5
-474 676 3
-936 1163 5
-600 29 2
-417 674 2
-184 13 3
-532 357 5
-694 965 4
-149 340 4
-345 1008 3
-109 28 3
-417 44 2
-144 713 4
-401 566 5
-293 746 3
-916 85 2
-58 181 3
-239 312 2
-272 134 5
-399 455 4
-157 118 2
-232 81 5
-332 210 5
-197 210 5
-807 141 3
-716 173 4
-18 692 3
-401 278 4
-543 252 3
-10 60 3
-339 1 5
-484 257 5
-535 735 5
-217 586 2
-164 685 5
-904 451 4
-894 179 5
-397 324 2
-889 1139 1
-851 121 4
-405 684 3
-746 1 4
-593 196 5
-130 876 4
-376 275 5
-476 399 3
-603 56 4
-26 748 1
-844 222 3
-566 480 4
-709 28 5
-429 441 3
-707 378 3
-239 1056 5
-870 663 3
-200 660 3
-523 629 5
-279 120 1
-216 416 3
-348 596 4
-658 730 3
-747 162 5
-758 217 2
-82 1001 1
-4 301 5
-276 820 3
-327 79 3
-312 191 5
-457 980 4
-807 510 5
-373 99 5
-786 708 4
-198 89 5
-455 1028 2
-639 306 4
-389 72 3
-394 419 5
-89 731 3
-892 273 4
-375 183 5
-209 129 2
-562 230 1
-216 67 3
-267 715 4
-755 301 3
-244 790 4
-643 208 5
-898 272 4
-836 896 3
-461 50 3
-181 983 2
-666 744 3
-388 218 5
-450 381 2
-548 340 1
-805 661 4
-851 1376 2
-892 237 4
-444 251 5
-927 738 3
-571 144 2
-280 128 3
-380 302 5
-534 370 4
-117 156 4
-6 276 2
-308 1252 3
-72 182 5
-788 431 2
-470 288 4
-608 306 4
-871 1345 3
-445 271 1
-508 1135 3
-815 215 5
-347 410 5
-246 385 1
-221 227 3
-569 14 4
-239 69 1
-276 748 3
-296 180 5
-422 271 3
-328 511 4
-37 273 3
-334 224 2
-771 216 5
-840 496 5
-121 508 4
-921 763 3
-765 50 2
-293 100 4
-796 39 3
-343 77 3
-715 235 2
-478 137 4
-328 911 3
-401 173 3
-151 170 5
-310 294 1
-58 773 4
-514 169 5
-642 1415 4
-114 527 3
-577 31 4
-445 479 3
-521 72 3
-697 222 4
-592 1017 4
-21 995 2
-27 1017 4
-650 576 1
-932 357 5
-846 31 4
-697 294 4
-615 529 5
-667 318 5
-332 651 5
-927 8 4
-537 657 3
-373 401 4
-534 125 3
-617 498 3
-936 904 5
-891 285 5
-119 405 4
-178 302 4
-708 751 4
-276 684 4
-16 448 5
-655 396 2
-354 175 5
-425 301 4
-690 153 5
-62 129 3
-460 7 3
-174 476 4
-727 207 5
-634 274 3
-626 988 1
-40 879 2
-711 747 4
-770 924 5
-99 694 1
-930 651 3
-521 33 4
-936 274 3
-284 305 4
-18 528 4
-94 1206 3
-890 662 3
-189 1400 3
-916 356 3
-846 519 4
-215 496 5
-504 934 4
-59 496 4
-842 886 4
-116 185 3
-733 100 5
-351 748 4
-883 137 5
-536 163 5
-409 945 3
-429 7 2
-10 216 4
-804 1615 4
-561 1229 1
-629 197 5
-766 510 3
-551 926 2
-632 182 3
-708 151 4
-87 1118 3
-484 258 5
-405 520 2
-429 381 3
-734 498 4
-521 203 3
-896 1672 2
-542 423 4
-249 1016 3
-174 284 4
-666 544 4
-292 528 5
-55 257 3
-92 655 4
-290 527 4
-545 199 4
-881 179 5
-786 100 4
-231 300 4
-577 399 4
-524 748 2
-689 597 4
-576 257 4
-876 435 4
-329 705 3
-751 323 1
-561 1230 3
-912 246 2
-102 233 3
-786 275 4
-796 222 5
-668 554 3
-95 58 3
-931 290 2
-592 23 5
-748 210 3
-60 510 5
-100 879 4
-701 124 5
-498 271 2
-334 606 5
-405 453 3
-587 326 3
-222 78 1
-119 237 5
-458 285 4
-345 535 3
-239 10 5
-308 60 3
-405 213 2
-379 405 3
-344 367 5
-435 380 3
-822 539 2
-472 763 4
-195 496 4
-633 110 3
-378 1028 2
-781 69 3
-201 124 3
-269 464 3
-314 787 2
-527 283 4
-711 660 5
-43 17 3
-49 173 3
-690 70 2
-94 553 3
-128 418 4
-521 181 4
-745 483 1
-370 604 4
-48 323 3
-387 83 4
-276 70 4
-573 661 4
-769 120 1
-784 292 4
-73 152 3
-336 1446 1
-178 66 4
-450 812 4
-506 77 3
-854 505 4
-479 405 4
-18 627 3
-551 1253 2
-339 74 4
-186 1033 3
-325 211 3
-737 173 4
-108 275 5
-181 413 2
-148 501 4
-94 86 5
-846 288 4
-315 303 4
-561 135 4
-364 1048 5
-465 134 4
-174 196 5
-768 354 3
-692 287 3
-297 699 4
-268 53 3
-698 205 4
-647 427 4
-43 64 5
-106 684 4
-863 302 4
-360 222 2
-345 684 4
-357 294 4
-537 653 4
-904 90 2
-398 135 3
-654 689 3
-394 552 3
-279 1485 4
-44 214 5
-533 196 4
-435 562 5
-871 182 5
-2 280 3
-268 298 3
-75 546 3
-426 178 4
-383 9 5
-293 232 2
-773 268 4
-868 556 3
-416 990 2
-15 742 2
-864 210 4
-479 272 4
-596 323 4
-504 322 4
-286 325 1
-6 502 4
-177 210 4
-807 99 5
-99 312 2
-606 191 5
-913 89 5
-683 271 3
-16 288 3
-795 768 3
-916 578 1
-897 660 4
-329 39 2
-3 323 2
-694 138 3
-939 1051 5
-379 192 4
-660 1135 2
-313 1470 1
-492 492 4
-524 82 4
-727 186 5
-851 56 5
-666 1047 3
-498 237 2
-627 470 3
-85 301 4
-387 50 5
-694 520 5
-213 692 4
-635 269 5
-460 312 4
-632 739 3
-757 62 3
-870 235 3
-48 988 2
-533 462 2
-498 54 2
-764 1 4
-742 15 4
-286 13 2
-864 423 5
-870 328 3
-576 514 5
-391 31 2
-548 443 4
-406 172 5
-25 501 3
-791 328 4
-266 275 5
-457 156 5
-727 356 3
-334 220 3
-193 94 3
-298 98 4
-28 173 3
-588 69 2
-474 511 5
-771 403 4
-213 182 4
-53 284 2
-432 1 2
-792 118 2
-902 423 4
-344 568 5
-59 760 2
-722 148 3
-297 204 3
-500 89 4
-258 323 4
-374 247 1
-395 216 3
-749 746 5
-711 71 3
-828 903 4
-194 580 4
-773 251 3
-91 313 4
-403 405 5
-703 1 4
-394 546 4
-644 871 4
-497 176 4
-436 159 4
-764 756 3
-629 162 5
-577 203 3
-419 300 4
-417 628 3
-87 47 3
-545 176 4
-504 196 4
-109 196 4
-644 977 4
-487 291 3
-472 367 5
-450 417 4
-785 995 3
-308 309 1
-489 338 3
-429 231 2
-64 48 5
-110 332 3
-406 184 2
-642 975 2
-648 164 4
-409 181 4
-641 59 4
-892 274 4
-655 611 3
-396 237 4
-619 578 4
-18 707 3
-184 780 4
-130 542 3
-286 72 4
-639 584 2
-755 880 4
-854 744 2
-851 204 4
-271 763 3
-150 150 3
-655 164 2
-896 518 3
-666 174 3
-749 419 5
-279 1088 4
-663 280 3
-761 1012 1
-854 100 5
-514 95 4
-57 926 3
-625 647 4
-44 96 4
-854 1013 1
-1 214 4
-110 231 1
-59 203 4
-116 902 2
-94 458 4
-287 276 4
-506 568 5
-880 177 5
-299 222 2
-222 895 4
-448 286 2
-488 288 2
-919 253 3
-894 30 4
-486 286 2
-650 491 3
-299 955 4
-495 84 3
-448 327 2
-374 758 1
-327 173 4
-882 369 3
-448 269 5
-344 274 2
-391 188 3
-653 196 2
-561 87 3
-846 1018 4
-626 879 1
-391 651 5
-454 988 2
-805 38 3
-307 145 4
-417 82 4
-64 81 4
-823 101 3
-409 165 4
-49 652 5
-303 219 5
-94 217 4
-94 763 3
-881 90 3
-851 109 4
-430 19 5
-807 597 4
-62 213 4
-650 742 3
-878 451 2
-660 1419 1
-494 748 1
-455 435 4
-450 388 3
-933 156 4
-64 1140 1
-346 17 1
-694 435 4
-602 261 3
-181 886 1
-457 470 5
-559 188 5
-194 357 4
-30 135 5
-472 419 4
-779 284 3
-796 91 2
-722 100 4
-250 678 2
-533 12 4
-861 275 5
-11 107 4
-636 275 3
-916 281 3
-10 603 5
-7 231 3
-457 1012 4
-334 230 4
-796 199 3
-882 275 5
-634 258 4
-87 401 2
-885 97 5
-638 144 5
-896 139 2
-418 362 1
-405 940 1
-421 879 4
-551 366 5
-758 684 4
-450 419 5
-537 959 3
-693 130 1
-184 559 3
-224 22 5
-457 7 4
-886 833 5
-239 504 4
-198 248 3
-876 174 4
-660 1178 1
-927 819 3
-679 268 4
-161 50 2
-763 518 4
-660 41 1
-916 91 4
-146 340 4
-279 92 4
-359 118 3
-489 883 2
-300 243 4
-449 544 3
-332 1150 3
-610 56 3
-416 78 2
-280 92 3
-122 28 4
-719 378 4
-662 1511 4
-479 204 4
-436 941 4
-73 48 2
-698 707 2
-399 1228 3
-527 659 4
-933 654 4
-868 433 4
-794 275 4
-862 820 4
-286 111 5
-711 316 4
-707 866 2
-315 164 4
-445 248 1
-146 294 1
-194 820 1
-102 13 3
-130 930 3
-180 1046 2
-715 71 3
-719 118 2
-437 419 5
-326 646 2
-515 329 2
-846 552 4
-56 408 4
-286 285 1
-499 879 3
-253 742 4
-651 332 3
-494 194 4
-13 852 1
-292 228 5
-592 236 3
-246 101 2
-923 168 5
-463 127 5
-435 52 5
-10 698 4
-826 101 5
-514 209 3
-1 208 5
-201 172 5
-483 151 2
-239 39 5
-500 1048 3
-294 603 5
-449 983 2
-125 173 5
-121 9 5
-472 7 5
-330 80 2
-308 522 3
-492 511 5
-833 226 3
-337 636 4
-667 694 4
-796 22 4
-405 1519 2
-747 900 5
-896 79 5
-121 174 3
-429 673 3
-221 327 4
-343 147 4
-484 230 5
-536 432 4
-472 323 4
-773 170 5
-655 197 3
-457 234 5
-567 156 5
-305 286 4
-303 1012 4
-828 730 3
-606 28 4
-299 313 3
-323 294 3
-72 135 4
-327 257 2
-593 762 4
-44 211 4
-11 737 4
-655 702 2
-294 1132 4
-851 564 3
-405 1030 1
-896 684 4
-807 678 3
-654 124 4
-429 425 3
-846 1050 4
-938 841 3
-74 15 4
-427 688 5
-13 182 5
-943 1067 2
-346 1025 3
-554 69 5
-445 87 3
-436 95 4
-817 222 4
-757 385 3
-385 1159 4
-216 763 4
-81 456 1
-378 88 4
-7 389 4
-851 455 3
-895 597 2
-142 358 2
-405 66 5
-246 567 5
-60 23 4
-96 423 5
-592 423 5
-95 878 1
-743 100 5
-682 185 4
-347 260 1
-59 7 4
-560 546 2
-393 82 4
-142 134 5
-901 111 3
-620 416 4
-535 515 3
-567 478 5
-825 696 3
-16 294 4
-86 1175 5
-833 1427 3
-479 692 3
-601 135 4
-256 1114 4
-618 521 2
-416 783 3
-537 28 3
-290 385 4
-843 265 3
-59 654 4
-354 59 5
-168 409 4
-318 393 5
-246 172 5
-130 342 3
-798 801 3
-721 82 4
-378 186 3
-682 824 1
-401 181 3
-280 218 4
-554 845 3
-256 665 4
-925 948 2
-60 435 4
-406 962 4
-406 520 4
-16 423 5
-486 220 3
-152 1054 1
-542 523 4
-37 597 5
-401 707 2
-887 756 5
-343 88 4
-135 203 4
-846 738 4
-269 959 5
-294 324 4
-85 98 4
-840 653 5
-666 97 4
-911 480 4
-415 1524 5
-881 14 1
-795 202 3
-622 1078 3
-265 50 2
-617 448 3
-364 321 2
-234 1149 3
-921 25 3
-846 426 1
-316 678 1
-573 513 4
-145 559 2
-239 443 5
-237 174 4
-276 1157 2
-854 124 5
-330 403 5
-311 199 4
-634 741 3
-145 308 2
-207 527 4
-621 161 3
-844 2 4
-545 31 4
-621 174 3
-406 204 5
-592 305 4
-264 12 5
-830 968 4
-627 939 3
-450 716 4
-18 152 3
-293 491 4
-174 577 1
-312 194 4
-24 176 5
-13 315 5
-507 319 3
-264 217 3
-840 609 4
-592 432 1
-299 512 4
-524 461 3
-588 832 1
-666 48 4
-648 167 4
-458 939 4
-927 411 4
-269 1427 2
-806 254 3
-807 403 4
-731 462 5
-845 269 4
-346 153 3
-722 310 4
-412 724 4
-130 235 4
-164 331 5
-92 108 2
-474 606 3
-605 237 3
-692 412 4
-242 294 4
-401 280 2
-145 760 2
-247 7 4
-641 258 3
-305 210 3
-392 837 5
-592 127 5
-327 715 4
-7 650 3
-334 882 3
-177 11 4
-70 69 4
-348 819 4
-313 216 4
-373 403 3
-786 703 3
-216 79 4
-347 168 5
-893 928 3
-820 316 3
-14 276 4
-85 697 3
-222 276 5
-440 512 3
-447 156 5
-537 684 3
-307 228 5
-207 194 4
-271 614 4
-425 879 2
-236 546 4
-484 451 4
-345 724 5
-724 268 4
-254 230 4
-141 1244 3
-749 62 3
-244 1017 4
-548 924 3
-593 40 1
-796 1012 3
-897 88 4
-895 284 3
-544 331 3
-642 1136 4
-311 179 2
-651 285 4
-899 283 4
-201 387 2
-537 461 3
-436 327 5
-328 199 4
-435 366 2
-683 316 4
-264 184 5
-363 117 5
-188 191 3
-479 185 4
-327 131 4
-452 660 4
-13 160 4
-481 42 3
-118 53 5
-823 125 4
-796 185 4
-393 278 4
-54 405 4
-472 185 5
-32 181 4
-222 215 4
-561 708 3
-533 477 4
-763 224 5
-537 746 3
-881 514 4
-466 333 4
-183 356 3
-399 29 3
-420 1347 3
-288 97 4
-430 744 3
-82 357 4
-363 707 3
-488 22 4
-864 401 4
-284 347 5
-387 151 3
-378 623 3
-92 720 3
-293 230 2
-708 147 4
-870 462 4
-503 269 5
-559 550 4
-551 1028 4
-253 647 3
-717 326 3
-416 1139 3
-276 235 4
-541 140 5
-496 1401 3
-603 747 3
-737 137 5
-11 730 3
-311 484 4
-38 404 5
-880 27 3
-396 823 2
-327 1067 4
-505 259 3
-423 471 3
-645 434 4
-194 79 3
-478 100 5
-514 168 4
-661 121 2
-445 458 2
-378 722 3
-450 357 5
-279 265 5
-897 185 5
-284 269 4
-405 714 1
-529 292 4
-788 561 3
-851 828 2
-184 202 3
-591 194 4
-330 385 5
-92 95 3
-4 360 5
-504 357 4
-577 5 4
-606 491 4
-589 678 4
-526 7 4
-99 315 4
-537 321 3
-417 779 2
-385 503 3
-608 673 4
-130 200 5
-586 468 3
-299 482 4
-798 81 3
-468 100 5
-881 141 3
-279 239 4
-734 172 4
-58 238 5
-650 566 3
-257 307 4
-416 942 4
-835 654 5
-207 748 3
-567 1 3
-886 1435 3
-551 572 1
-557 288 1
-392 23 5
-593 1014 1
-378 52 5
-864 136 4
-452 318 5
-308 569 3
-450 121 3
-437 755 3
-758 29 3
-586 27 3
-319 306 4
-27 118 3
-174 100 5
-919 100 5
-222 724 3
-15 932 1
-643 82 3
-321 498 5
-570 243 1
-70 542 2
-549 100 4
-537 1008 2
-110 64 4
-394 343 3
-699 413 3
-892 71 3
-565 1018 5
-456 1198 4
-70 217 4
-409 59 5
-379 176 5
-436 1227 2
-224 286 3
-497 588 4
-261 596 2
-378 977 3
-262 235 2
-387 1537 4
-935 620 2
-922 660 3
-919 527 4
-141 276 1
-610 527 4
-95 509 4
-487 56 4
-750 286 4
-345 678 2
-10 518 4
-297 116 4
-220 298 4
-711 111 2
-832 245 3
-669 313 4
-548 1278 4
-202 179 1
-650 290 2
-263 269 4
-312 528 5
-206 313 5
-532 1226 4
-244 1118 4
-58 1098 4
-606 531 5
-276 1135 4
-932 495 5
-57 243 3
-897 82 5
-278 882 3
-286 946 3
-896 800 3
-645 188 4
-7 676 3
-435 929 2
-117 406 3
-239 180 5
-487 541 3
-387 659 4
-318 15 5
-497 578 4
-267 174 5
-655 285 4
-293 31 2
-747 208 5
-731 197 5
-416 980 4
-727 369 2
-201 436 3
-5 448 2
-622 41 3
-416 100 5
-588 739 4
-934 82 4
-865 919 5
-684 239 4
-618 282 3
-633 423 4
-387 461 5
-60 511 4
-343 52 5
-880 70 4
-680 121 3
-102 396 2
-5 209 5
-43 95 4
-682 217 4
-854 58 3
-125 85 3
-90 1134 3
-139 744 5
-286 930 2
-773 6 3
-347 204 4
-293 843 3
-90 478 5
-13 858 1
-344 864 3
-395 163 5
-347 879 3
-448 262 4
-126 884 5
-921 284 4
-554 181 4
-913 301 1
-13 905 2
-624 271 3
-13 377 1
-655 736 3
-788 176 5
-537 279 2
-95 597 3
-557 508 4
-425 177 3
-198 1117 3
-702 748 2
-343 177 4
-291 77 4
-682 1478 3
-655 1208 3
-734 56 1
-611 752 5
-235 175 4
-329 338 2
-804 194 4
-526 325 3
-313 696 3
-13 29 2
-160 276 5
-416 202 4
-90 237 4
-660 222 2
-118 547 5
-214 482 4
-195 636 2
-709 234 5
-693 606 4
-291 941 4
-751 50 5
-85 194 4
-361 514 5
-177 221 3
-458 603 4
-936 1160 5
-562 50 5
-90 962 2
-756 550 2
-803 690 4
-405 80 1
-586 161 5
-30 403 2
-608 448 5
-455 591 4
-406 101 3
-533 568 5
-592 1129 5
-435 23 4
-450 98 4
-65 63 2
-597 235 4
-833 121 1
-527 651 5
-833 23 5
-25 125 5
-388 508 3
-864 673 3
-194 568 2
-457 529 4
-429 1074 3
-339 427 5
-295 380 4
-634 121 5
-160 952 4
-329 303 4
-790 401 4
-561 10 3
-508 195 3
-798 432 4
-251 978 2
-72 402 4
-608 197 5
-85 663 5
-848 517 5
-183 331 3
-224 323 3
-130 367 4
-48 480 4
-911 185 5
-200 501 4
-484 399 4
-489 304 3
-521 1014 3
-213 504 5
-497 951 2
-82 367 4
-276 1210 2
-369 900 4
-57 288 4
-258 288 1
-131 251 5
-805 191 4
-346 748 4
-284 324 3
-757 193 4
-24 58 3
-389 124 4
-826 779 3
-760 50 3
-490 50 5
-899 154 5
-543 730 3
-394 672 3
-173 245 4
-425 877 3
-85 157 3
-606 121 4
-886 65 3
-933 105 2
-697 7 5
-496 659 3
-934 384 4
-864 283 5
-234 661 5
-608 1119 5
-543 1194 4
-285 270 4
-145 249 4
-664 96 3
-798 795 3
-327 333 2
-336 742 3
-416 90 4
-151 234 4
-669 508 3
-925 567 3
-496 88 1
-25 86 4
-503 702 2
-749 38 3
-846 720 4
-674 929 3
-911 176 4
-924 433 5
-322 521 5
-374 148 4
-232 630 3
-365 237 3
-709 187 5
-204 880 2
-249 11 5
-561 597 3
-846 387 3
-487 96 5
-757 148 4
-862 982 4
-71 154 3
-896 1471 1
-185 275 4
-896 928 3
-85 419 5
-660 118 2
-642 946 2
-334 193 4
-13 2 3
-711 255 4
-357 1 5
-640 689 4
-500 531 3
-291 181 5
-87 201 2
-99 12 5
-918 275 4
-256 220 3
-57 250 3
-243 1197 4
-10 160 4
-493 210 5
-7 627 3
-560 240 3
-239 512 5
-698 222 4
-331 694 4
-532 353 2
-532 1199 3
-627 673 2
-705 161 5
-7 678 3
-92 210 4
-806 45 4
-457 385 4
-459 989 5
-722 25 4
-521 743 1
-758 1142 5
-10 176 4
-537 48 4
-116 269 3
-608 133 4
-198 939 3
-806 168 4
-693 1090 4
-455 252 3
-723 169 4
-894 165 4
-508 1067 4
-406 498 5
-82 1033 1
-318 795 2
-119 86 4
-103 294 4
-828 275 3
-95 282 4
-618 1212 2
-249 1167 4
-276 649 4
-815 472 1
-871 172 5
-639 179 1
-291 798 4
-811 901 4
-287 156 5
-64 141 4
-89 949 3
-276 1028 3
-389 479 4
-586 566 3
-450 686 4
-417 450 2
-21 680 1
-313 15 2
-474 141 4
-796 1001 2
-657 475 4
-303 156 5
-833 665 3
-763 730 5
-374 181 3
-500 662 2
-330 67 4
-741 194 4
-590 754 3
-592 342 2
-452 202 3
-265 815 3
-422 379 2
-851 9 4
-378 49 3
-660 1078 2
-406 4 2
-200 191 5
-561 222 3
-310 304 5
-63 288 3
-338 525 4
-200 313 5
-399 203 4
-201 45 2
-489 353 4
-269 508 4
-275 181 4
-402 1060 3
-665 421 4
-708 326 4
-215 70 3
-417 12 4
-194 518 4
-198 127 5
-883 234 4
-101 405 4
-207 273 4
-435 470 2
-933 453 1
-405 57 1
-470 950 3
-721 331 3
-837 289 5
-314 756 3
-13 771 3
-921 202 4
-555 302 3
-698 

<TRUNCATED>
http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/test/resources/hivemall/mf/ml1k.test.gz
----------------------------------------------------------------------
diff --git a/core/src/test/resources/hivemall/mf/ml1k.test.gz b/core/src/test/resources/hivemall/mf/ml1k.test.gz
new file mode 100644
index 0000000..346b858
Binary files /dev/null and b/core/src/test/resources/hivemall/mf/ml1k.test.gz differ



[5/6] incubator-hivemall git commit: Close #13: Implement Kernel Expansion Passive Aggressive Classification

Posted by my...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/test/resources/hivemall/anomaly/cf1d.csv
----------------------------------------------------------------------
diff --git a/core/src/test/resources/hivemall/anomaly/cf1d.csv b/core/src/test/resources/hivemall/anomaly/cf1d.csv
deleted file mode 100644
index 6090026..0000000
--- a/core/src/test/resources/hivemall/anomaly/cf1d.csv
+++ /dev/null
@@ -1,2503 +0,0 @@
--0.251822346366
-0.168460024955
-0.0326258118256
-0.323858249314
-0.325848830384
--0.225750529039
-0.234520559066
-0.077308200191
-0.385776562964
--0.168098455482
-0.346208079804
--0.0754811998138
-0.901941680326
-0.247384411994
-0.019821467936
--0.344238099414
-0.179556329173
--0.0115880931648
-1.11570123355
--0.279934539731
-0.31649639898
-0.0747070508132
-0.145242555261
--0.11877518672
--0.119189762376
--0.264323772748
-0.123616073597
-0.574664860754
--0.25688247129
--0.0556179482527
--0.0354601572347
--0.16578225106
--0.552305930496
--0.424647065466
--1.07478236025
--0.128231576278
--0.218602393052
-0.302126003135
-0.111319830334
-0.36923322137
-0.0725741448463
-0.361590407865
--0.289153695373
-0.34139190129
--0.200251895506
-0.220222114893
-0.208678305877
--0.596416510438
-0.0675530663433
-0.422358869069
-0.336020796447
-0.0980319619742
-0.678812643879
-0.506480176083
--0.042002113051
--0.0683007841989
--0.1295608625
-0.126826999575
--0.636131762155
-0.329331949087
--0.0559525930785
-0.11582026579
--0.358895003811
--0.240394636584
--0.264775468001
--0.163502959619
-0.322703220973
-0.122018902228
--0.0674522054942
-0.301073421587
-0.122823973447
--0.37730819037
-0.190476720254
-0.587135991371
-0.66703532485
--0.461091271889
-0.255409286131
--0.296645899082
--0.540193856776
-0.799168818292
-0.496557610606
-0.253872904315
-0.456383447415
--1.08206833229
-0.534612666633
-0.231601383737
-0.403764872684
-1.00162594269
-0.646531101831
-0.148252077601
-0.0611175461773
-0.155653104555
-0.32593041174
--0.248162228572
-0.375224953329
--0.0613759154201
--0.89944279458
-0.0424643049425
-0.170072494114
-0.300418184654
-0.0758735664997
--0.561038512753
-0.618146763147
-0.0356279549583
--0.326883999555
-0.289271696328
--0.00522767201368
-0.662231269907
-1.24009352597
--0.299823796947
--0.364882707587
-0.00765983734197
-0.162935368309
-0.580379417616
-0.179871330841
--0.0962210607872
-0.229645627994
--0.837949127013
--0.0472134542254
-0.0582675435528
--0.264328265065
-0.35316949172
--0.4493386479
--0.887507285041
-0.861776797311
-0.687823437557
-0.395146935456
--0.227937100638
-0.0511597155782
-0.456368539424
-0.22845718298
--0.142486022776
--0.190904383799
-0.489370282688
-0.0269291500038
--0.290085512301
-0.227285849482
-0.0244225926493
--0.142887115622
-0.599701776858
-0.170382179229
--0.0690624349227
--0.681639010915
--0.0436566487536
-0.0181584537093
--0.509785458253
-0.588390875139
-0.394193453312
--0.600548682804
--0.165435134249
-0.0133589899798
-0.688692807534
-0.401553709377
--0.262132070713
--0.303086338913
-0.421647957045
--0.413635790025
-0.0215119629478
-0.76619711569
-0.324549900584
-0.192828733753
--0.00255587237884
-0.197542906396
-0.126752926246
-0.220967704965
-0.0529165419099
-1.00635167501
--0.144536892738
--0.340098198867
--0.252300685772
--0.751690716725
-0.119898456233
--0.284765461587
--0.151763068097
--0.463672423403
-0.258281025918
--0.0177408005525
--0.0881536502335
-0.131593710877
-0.676368632432
--0.297419571762
--0.0785042171049
-0.0809803145183
-0.094200026839
-0.811917752542
--0.302851610323
-0.0356841866293
-0.293291709352
-0.260560026864
-0.414469508052
--0.000282988601909
-0.0233808737274
-0.0939400119811
-0.00513869483717
-0.800921875769
--0.0699541656893
--0.517464070655
-0.55205965687
--0.0824700180974
--0.228219467093
-0.224349970879
-0.520431836497
--0.429257083384
--0.0201330513663
-0.171746263913
-0.276825728331
--0.743382970644
--0.212841530984
-0.450144802241
--1.01057777174
-0.729880721498
-0.0556983421819
-0.0639091378551
-0.0458902866755
-0.766656127381
-0.00315081094191
-0.0803055933283
-0.137640146678
-0.226544839401
--0.641403139313
--0.438129904673
-0.0647928457392
-0.165296828338
--1.00080786949
-0.523401334733
-0.401044548829
-0.222678273511
-0.659589546187
-0.181467455181
--0.347867490389
-0.535220769791
-0.284746708422
-0.585191858361
--0.423474739732
--0.1749697269
-0.174947600159
-0.282855222063
--0.026916973951
-0.216476645446
--0.592862836299
-0.414440514035
-0.17419329994
-0.00997259744621
-0.16987331928
--0.561508482399
--0.0702102951069
--0.40347932993
-0.286762548559
-0.472227637024
-0.383489657445
--2.56649253378
--2.52154391175
--2.73805323286
--2.5877064734
--2.5742997091
--2.66512384735
--2.74503052556
--2.7336020658
--2.59684809905
--2.60318314912
--2.74575178315
--2.68964624301
--2.5821753898
--2.70646730498
--2.68405422295
--2.55133681972
--2.60832012581
--2.60169726077
--2.67619032903
--2.7073680964
--2.46497336094
--2.5271867013
--2.74051700448
--2.68645653841
--2.60958639569
--2.63933427351
--2.63993696975
--2.60292261103
--2.7246870946
--2.84956256595
--2.53533426013
--2.5663686546
--2.59283018198
--2.62034799096
--2.67435984736
--2.62125812133
--2.69039246805
--2.60801265524
--2.50377036779
--2.58925153836
--2.58388519831
--2.70029751758
--2.68142892291
--2.69297072231
--2.52496443817
--2.72772044115
--2.69362970838
--2.62871826027
--2.66091848664
--2.70654715919
--2.65172100076
--2.62700108509
--2.65032886103
--2.59259289458
--2.60528100969
--2.62285834849
--2.80785395133
--2.57877399765
--2.64182240692
--2.80951914545
--2.56618282409
--2.74157007783
--2.6948943037
--2.57092580328
--2.51215447765
--2.56909875024
--2.66232951123
--2.66914089763
--2.65936182964
--2.74888383208
--2.57955221196
--2.65327857095
--2.57908562617
--2.66225420683
--2.5138595862
--2.53158264505
--2.83725085683
--2.52955892758
--2.6353064055
--2.72130046821
--2.6332374054
--2.72364146545
--2.64485531669
-0.0
--2.5484819522
--2.74079435253
--2.75259140225
--2.74097153511
--2.69077819636
--2.65945373654
--2.68200622657
--2.64568349869
--2.59896827855
--2.75557596667
--2.58656022443
--2.64930123272
--2.70668407368
--2.59049608371
--2.71376995427
--2.82483801165
--2.64822292463
--2.66826785711
--2.74017387492
--2.80274815057
--2.66629434376
--2.55858041378
--2.49138584807
--2.58355757839
--2.64141296465
--2.6897086003
--2.66979998174
--2.72789966098
--2.7275805659
--2.61177553558
--2.80082761177
--2.66468251088
--2.67238594829
--2.56071280895
--2.61418371517
--2.64661883344
--2.6316899444
--2.49940123308
--2.56333278338
--2.58241561864
--2.61627566047
--2.63820152832
--2.5917965686
--2.5905991971
--2.68412228134
--2.64769508242
--2.74032544786
--2.71537164359
--2.67046079116
--2.74241612192
--2.61751784851
--2.62220046335
--2.68581674998
--2.61645782938
--2.7891632518
--2.61445736757
--2.80957984258
--2.73520987645
--2.61058087145
--2.62677000121
--2.64816674655
--2.63589286039
--2.66980484616
--2.62286657422
--2.62507502043
--2.70238033833
--2.75203803324
--2.70249173454
--2.71084730729
--2.73089611654
--2.53835117577
--2.68605351947
--2.51834563071
--2.58384625648
--2.62028772813
--2.46375945005
--2.57658741384
--2.69788738918
--2.66815316579
--2.69239255562
--2.6126408738
--2.54334903904
--2.67789229175
--2.6232883405
--2.72963990466
--2.67493774544
--2.58900431157
--2.56010179092
--2.65087426192
--2.58066814354
--2.60304211781
--2.71030651737
--2.50327859297
--2.63823665711
--2.75720624347
--2.75278761735
--2.72869456648
--2.58381266312
--2.63071914245
--2.65522840696
--2.82529855606
--2.59846485309
--2.65998916093
--2.6404951945
--2.54236187243
--2.58655981393
--2.57692618709
--2.57743282341
--2.6296510284
--2.70922842252
--2.60559606371
--2.73980146834
--2.55101650116
--2.71622837758
--2.57931734585
--2.79570209211
--2.64555809698
--2.67225219282
--2.5352084289
--2.5679552064
--2.66544973315
--2.62608931388
--2.54447996497
--2.70869475811
--2.86064270456
--2.72753048058
--2.73642242508
--2.7630886908
--2.68078903078
--2.68972041088
--2.67070637352
--2.65273857424
--2.55910497272
--2.67187344045
--2.5990642422
--2.5737661116
--2.80561084885
--2.52940649296
--2.68713445397
--2.69570596131
--2.62735909128
--2.67258076449
--2.75623157401
--2.68265870303
--2.78091477072
--2.72882205158
--2.58142277531
--2.70282391224
--2.72677107868
--2.7618656547
--2.7014764018
--2.740331912
--2.71321187474
--2.63463189155
--2.70148859614
--2.67617018906
--2.67955749603
--2.59497274189
--2.51384401325
--2.72571034603
--2.75345898853
--2.66001402963
--2.65769909355
--2.65912911341
--2.62146589224
--2.71136467508
--2.78146800263
--1.09580583392
-1.71195963652
-1.78754286548
-1.97274694721
-1.29881586709
--0.362457319175
-1.10117380407
--0.615363209108
-0.957083782431
--0.525081048032
-0.733527964003
-1.25877845141
--0.545618002607
-0.474244784522
-0.169195665294
-1.06400876115
-0.8694266983
--0.464826148106
-0.512823106948
-1.38715403122
-1.15048714847
--1.3725144052
-1.17142504556
--0.648486558008
-2.52783790583
-0.0150607051189
--0.842534215391
-0.399131548584
--0.690599995776
--0.788519732164
-2.62732575151
--0.281546847452
--0.912271031121
-2.12914377269
-0.513275334545
-0.0814305907291
--0.252793700773
-0.54925630927
-0.701775991354
-1.07258952642
-0.759821657985
-0.830254859313
--0.140441694281
-0.973938012988
-1.20884158127
-1.36879065149
-0.722908132237
--0.219735218956
-2.66285689099
--0.370381080077
-1.26021117414
-0.75906756469
-1.76725596876
-2.72461800547
-1.5404408537
--0.223295843082
--0.017668079731
-0.750982621562
-1.23690401582
-0.714477152522
-1.75235099695
-1.21036333859
-0.900619935221
-0.25066175477
-0.510477856152
--0.117985851221
-0.910256622717
-0.983829527212
-1.19080841772
-0.267236287401
--0.299850001285
-0.556984585182
-0.539767196444
-2.15240095045
--0.0256629355225
-0.623654875952
-1.86454239589
-0.203963055099
-1.67391162434
-0.089548175744
-0.883394092827
-2.38049912948
--0.089659164471
-1.4200360216
-0.262226368689
-0.780531973304
-0.546512066876
--0.411768942708
--0.542899201688
--0.475247565078
-0.866583336896
--1.02554946437
--0.388463978523
-0.642800447325
--0.272947570755
-0.084460173233
-0.881280913126
-0.942768813628
-0.545748843759
-0.800892294402
-0.867210729261
-0.51899734487
-0.887412426446
--0.023711765116
-1.28677259244
-2.35539759868
-1.4799010959
-0.911913703705
-0.696484777254
-1.34343869447
-0.605156481219
-2.09144354177
-1.57575409562
-1.89287380912
--0.664182532271
-0.826055972135
-1.49141615528
-0.21752776122
-1.52846684678
-1.55030064083
-0.646612689813
-1.63834898212
-1.61433216461
-2.25271798805
--0.523543560354
-1.41245698531
-0.591610245459
-0.771710263487
-0.424987322849
-1.82832951513
-1.60940345918
-1.66071794434
-0.590028914761
-1.2417870388
-0.39022814812
-0.233954821677
--0.527559895363
--1.20485885618
--0.4164127596
-0.949538122655
--0.362943051114
--0.263408167629
-1.41730146789
-0.736914456633
--0.517882304478
--0.772493818159
-0.587129899113
-1.64347991057
--0.847128320234
-0.0596758338942
-1.53433348488
--0.949491134999
-2.08697584843
-1.88302003958
-0.292608584605
--0.472396612274
-0.271788709448
--0.285790607344
-1.16978972484
-0.685004349763
-0.0631408731559
--0.735807097135
-1.4127752112
--1.38426012768
--1.18601261156
--0.423126550027
--0.757788287328
-1.30933429447
-1.76915963761
-0.503247346329
-2.31941535653
-1.87049214267
--1.51946015511
-1.65670232359
-1.12169605651
-0.176652200765
-1.22619783596
-0.6101937698
-0.376435281953
-0.824765167208
-0.475383305332
-1.17028651406
--0.354590500625
--0.405652084497
--1.37463344225
--0.231484656057
--0.214450002142
-0.808080707747
-0.464716159295
-1.34823071604
-1.37631625567
-0.398864252737
-0.308500399045
--0.203212778885
--0.0862844608113
--0.00722568806972
--0.351410640688
--0.193352901955
-0.164791589527
-1.097650579
-1.78007963413
-2.23004158076
-0.991152481896
-1.560862265
-1.35417505776
-1.62058833819
--0.222987880708
-0.319867602638
-1.04549800961
--1.07886434731
-0.0191057484574
-1.32811460541
--0.657343622515
-0.00976308517434
-0.766379111236
-0.474740766785
-0.0581802953092
-0.156045764344
-1.31126392105
--0.467352660869
-1.13277922305
--0.680710711564
-1.14645921626
-0.289938460952
--0.993792262516
--1.12835023154
-0.287576085585
--0.375276982142
-1.42477710543
--1.05133172639
--0.322222423534
-0.215649096724
-1.00873023244
--0.182982715146
-0.195983038392
--0.216142654576
-0.831811716053
-1.93992354275
--0.360862121932
-0.0799090947132
-0.743714631499
--0.224583205032
-1.46094651896
--0.0234865591527
--0.185966315361
--0.0485231393337
-2.54381171383
-0.530144864346
-1.69674493057
-2.30764696664
-2.44028593308
-2.3638898133
-2.37727485136
-2.28239246417
-1.86890298578
-1.59937057241
-2.42461541987
-2.02750390266
-2.31605816413
-2.32267115986
-2.43047654599
-1.83488976621
-2.17919778455
-2.19169816405
-1.83392541197
-2.65706199167
-1.98437526476
-2.01064251333
-2.52220355387
-2.33303450234
-2.07049413919
-2.52433633829
-2.59021345236
-2.00712920785
-2.73124729658
-2.46505123066
-1.74119629814
-1.94695303927
-2.00394581099
-2.06088626693
-1.73466246811
-2.13677561632
-2.42188906239
-2.66400165002
-2.13892893806
-2.10448592977
-1.0315350357
-1.91821727951
-2.41138907411
-2.62839131198
-2.37832689778
-1.65144513906
-1.36955832257
-2.82521862733
-2.46385171442
-2.51717938141
-1.63892667799
-2.32509937479
-2.22842164597
-1.58979173793
-2.20691900355
-2.17783189087
-1.9591593475
-2.12581600219
-8.0
-2.40836152409
-1.86948034173
-2.00774969432
-2.4532148636
-2.54932386257
-2.13165074748
-1.91530064164
-2.78153427028
-2.36799480071
-2.55419048088
-1.99090772258
-2.37654395029
-2.09795108672
-2.31683958085
-1.82120796009
-1.97170706933
-1.55146690294
-2.99879672476
-1.99774313398
-2.95612041088
-2.1085327637
-1.90812283013
-2.45136734475
-2.57975973841
-2.47975980317
-2.36004428412
-2.19101866675
-1.90306964851
-1.85700810132
-2.46335887663
-2.39155527402
-1.54709209308
-2.28609136125
-1.66443764009
-1.89938137124
-3.05957156051
-2.08870368873
-2.36039026361
-2.81598513742
-2.39620635642
-2.43757478868
-1.65643933518
-3.07527932164
-2.08347916075
-2.9157514332
-1.84532854039
-2.59870040636
-2.63923953284
-2.21020404847
-2.63987801428
-2.34971881263
-2.32008606597
-2.49330616641
-3.03415354158
-2.19025049999
-2.25258103111
-2.0573048509
-1.58932208033
-2.57595797884
-1.75011979736
-1.99426931738
-2.64631897052
-2.48611582324
-1.72185348959
-2.27365389982
-1.8394564477
-1.86454595884
-1.93732005397
-2.5131562182
-2.49175432968
-2.51138984388
-2.24910156708
-1.94671813445
-2.04856617924
-2.4242466637
-1.72459915926
-1.74649274174
-2.39835630049
-2.23241141385
-2.04810022911
-2.08396698801
-1.90257085988
-1.40857278635
-1.59076934098
-1.71691194204
-2.86403841379
-2.35630191228
-2.15643625651
-2.06478148079
-1.66558560809
-1.8049047947
-1.799550809
-2.18469689165
-2.52113863978
-2.31425208727
-1.75254989201
-1.89784715559
-2.12700707995
-2.51235929765
-2.24022030768
-1.65201929525
-2.27208272405
-1.79908561002
-1.54950931536
-2.84272215024
-2.71878438518
-2.1111924984
-2.54959801149
-2.42101732299
-2.30332648544
-2.36593442404
-2.42381860785
-2.24230742935
-2.03489901481
-1.95289556854
-2.40718382406
-2.42886945934
-1.82210619215
-2.0289496571
-2.21169857287
-2.38678893387
-2.69835204521
-2.3818747123
-1.78776932268
-2.14330577217
-1.90786171005
-2.67418594521
-2.34417674619
-2.10009561421
-2.29223923814
-2.02569770722
-2.02758863077
-2.09976347755
-2.14409353979
-2.20418810721
-2.8905185953
-1.91915973236
-2.43124579807
-1.64979122527
-2.85884939336
-1.86955040765
-2.15951119604
-1.92783674158
-2.15498232215
-2.26493077618
-1.83851326674
-1.97328463931
-2.67183037698
-1.7499564159
-2.93979711279
-1.36288735491
-2.22195409034
-2.8380050875
-2.08881697514
-1.36768024614
-1.35381251554
-2.16539266185
-2.29525431521
-1.85642140895
-1.84842932149
-2.12796661401
-1.42649330899
-1.9296496412
-2.04877990182
-2.60234983795
-2.50480384711
-1.93785588933
-1.34062845149
-1.95576513868
-2.45165996906
-1.91372363076
-2.24368941499
-1.97834280359
-2.71167331291
-2.16226402493
-2.48162412445
-2.40064368562
-1.84838017938
-1.98259064914
-2.53865930893
-2.31612742176
-1.98083768598
-1.64670142703
-1.43461542505
-2.14488305969
-1.83095305377
-1.80856908232
-2.26121422482
-2.45361875289
-2.18185865742
-2.47096720815
-1.94456737403
-2.13492658894
-1.91993102677
-1.90983340601
-2.61567368951
-9.01935223129
-8.33955659127
-9.00967981846
-8.39574891516
-8.96734414785
-9.72256514217
-8.68707815626
-9.62841793555
-8.24864825685
-9.28897688785
-9.29750745169
-9.15438800322
-9.02986986755
-9.22778915143
-8.85463581932
-9.51753110714
-9.34323153261
-9.58599200221
-8.84432260169
-8.11619559089
-8.48764574669
-9.12975852128
-8.19358761424
-8.8812823162
-8.53917097364
-9.64296410694
-8.46965003246
-9.36458952692
-7.59990952337
-8.03461261742
-8.35816287808
-8.25807051826
-9.25961816459
-8.44281376891
-8.70800837524
-9.0295810722
-9.21441184379
-9.3519162205
-9.22558818796
-10.3165384727
-7.56912498497
-10.089720791
-8.73500055993
-9.36317802373
-8.73598718491
-9.1133405406
-7.99361965757
-8.81534662638
-8.69557490253
-9.63371633909
-9.57372704104
-8.43022876818
-8.44227703068
-8.90392947308
-9.27847757346
-9.65911077792
-8.65720316401
-9.32047667519
-9.01339349801
-8.76427440121
-8.10315012508
-9.27512234728
-8.69154299299
-8.83005147836
-9.79867469293
-10.3149304379
-8.48149899977
-7.78424756837
-8.31257836562
-9.22194903568
-9.44168949892
-9.40051108121
-9.16789852785
-7.92004468965
-9.77473584871
-9.51404501827
-8.60824073105
-9.23798007388
-8.20764140947
-10.1196088501
-8.33869834445
-9.63107090018
-8.66622128366
-8.10346781665
-8.69024763074
-9.29301107823
-9.41840207468
-8.96549377265
-9.79011711852
-8.7844394512
-8.60155104799
-8.87771865984
-7.20056779378
-7.85067383836
-8.64876132911
-9.83248533967
-9.33264874076
-10.0618855043
-9.20906864788
-8.46316111768
-9.31787156202
-9.37923008603
-8.70801890393
-9.37281853391
-8.45869945776
-8.75064739855
-8.65873755018
-8.85985757673
-9.56433639592
-8.68625467409
-8.94434016034
-7.65981604459
-8.28675855977
-7.7406356242
-9.16065872502
-9.28477612471
-8.71228827067
-8.6678119019
-8.65933947658
-8.6597159656
-8.78781580872
-9.14746415309
-9.48591442601
-8.42118230967
-8.722536502
-9.69944437
-8.41324722084
-9.28805890401
-9.15839541805
-9.76334744416
-8.74642649395
-9.20114722823
-9.36436173727
-8.04084578919
-7.94403387236
-8.53050580558
-9.55377384402
-9.33653661942
-8.89539841609
-9.64594025209
-8.13434675577
-8.35566858256
-9.7192774369
-8.70569671814
-8.58019341947
-8.15301760753
-9.24663488739
-7.5682768897
-9.16751819526
-9.18876397169
-9.22480634846
-8.2436240656
-9.20993059005
-8.31654716355
-9.12247624207
-8.02907960746
-8.20037825942
-8.53041905944
-8.14839561638
-9.45084574062
-9.12056202062
-8.85768711587
-9.03430926622
-7.61792802203
-9.45913543612
-9.3529156303
-9.40865622467
-7.99049074794
-8.41387262659
-9.32758547872
-9.50667343348
-9.09622526443
-8.55868274797
-9.2717175811
-8.68210339595
-8.624839672
-10.0959750331
-9.52986011235
-9.81773612626
-9.13363470443
-9.35303488395
-8.05396886882
-9.18431928403
-8.66732076251
-9.23261728122
-8.96728401864
-9.70167767414
-7.52634907718
-9.65694523583
-8.67756981556
-7.97643528888
-8.08738065985
-9.2287331606
-9.08802794039
-9.80208639997
-8.75888359485
-8.72967918129
-10.0097441237
-8.31963774927
-8.222204992
-8.13342465567
-8.3496477026
-9.89126005056
-8.58091492902
-9.24703407959
-10.0987320226
-8.80601416598
-9.5249203023
-10.0246346081
-10.0964996118
-7.73094252297
-10.4369674204
-9.27011644391
-9.23994971587
-9.00747237117
-9.05351279134
-8.45917498336
-9.23846326842
-8.34482963339
-8.56704529027
-8.4419632729
-9.24018531623
-8.32266846943
-9.11630549326
-9.41100723447
-10.1465334203
-8.60298431938
-8.58753727622
-9.49973615595
-8.81765673816
-8.86618545878
-8.91864262121
-8.88712021529
-9.23891701871
-8.59743249045
-7.79561074579
-8.8996677425
-9.02664197103
-9.49054108016
-9.10098071996
-9.08172789241
-7.85236621092
-8.88518310041
-8.98418541059
-7.69320752783
-9.11406585683
-8.24826906987
-10.4340133083
-8.63021345395
-8.94016472057
--1.64056337537
--1.1762173352
--1.0833209278
--1.84895505133
--2.22047826047
--1.92649493684
--2.65786716933
--0.903193192838
--0.965853794977
--3.18407903909
--2.37985115031
--2.27039879736
--0.938279115296
--1.50206904327
--1.99752016575
--0.993817242656
--1.32393466828
--0.874924123331
--1.19610214986
--1.625628658
--0.743232394024
--2.43703127191
--1.01867824556
--1.5510812509
--1.85813852685
--1.69036661544
--2.68018186953
--1.7848324259
--0.859958142252
--1.96863375744
--0.999205439016
--1.04906573168
--1.21010980997
--0.763328115998
--0.684969926735
--2.28062620944
--2.05534261163
--1.67083454632
--2.34101338262
--1.75893722756
--0.663585404519
--1.92308646881
--1.25318369002
--2.56317023425
--1.47340263609
--1.77834469634
--1.10148633833
--1.75304232639
--1.0108856366
--2.04008184606
--1.8324115794
--3.36358836221
--1.02386228897
--2.42548100631
--0.973037103012
--2.28661909146
--2.79655928338
--2.51019350083
--1.49715596207
--1.26052678372
--1.78493126259
--2.33197998845
--1.56197796785
--1.29231129204
--2.02151161614
--1.22555557687
--0.208073875837
--2.13636252811
--0.796067222979
--1.80099806111
--2.32854157868
--2.41631273348
--2.650168448
--1.75451642858
--2.22132467156
--2.23559420368
--1.13051161034
--0.73393087402
--1.7673900776
--0.76719820454
--1.37254420896
--1.51116826679
--3.4085150271
--1.3519002686
--2.74020351075
--1.92102228845
--0.878254390137
--0.550163762529
--1.57879911142
--1.86069374993
--2.2445025441
--1.5886024951
--2.84161330911
--2.16885245573
--2.6207482739
--0.879996003528
--0.763865951267
--2.0136396173
--1.22507096421
-0.885107698112
--2.95644609479
--1.35796550015
--1.41353858142
--0.524054304712
--1.88392070743
--1.36866745368
--1.10306846551
--1.89872257949
--1.28781097531
--2.32686491152
--1.20733793251
--2.92241795582
--1.83478778289
--1.98207302593
-0.0945833439636
--2.24093615561
--2.34862833529
--2.37212632371
--0.680959352366
--3.27883040946
--1.11975564099
--0.576827732546
--1.33046268784
--3.35703586461
--1.26708831756
--1.67402300317
--1.34521243668
--1.70824370561
--0.920572322436
--2.56384078539
--1.34094801452
--0.972720535887
--2.25874798875
--1.70570351225
--2.08585395194
--1.1945376583
--1.71745175461
--0.877527072367
--2.26488162339
--0.364159236404
--0.633812650198
--1.33497316771
-0.328288378087
--1.98372229387
--1.47670694367
--1.73700169926
--1.73675813809
--2.66978701731
--1.05638924395
--1.82833453092
--1.57258694807
--0.352139564249
--2.64827126846
--2.67561043773
--2.00723011224
--1.75015302963
--1.39962617755
--1.60915597084
--2.02593633687
--1.12334374199
--1.16860385521
--1.2162461225
--2.54004446218
--1.96018073167
--0.895626272555
--1.52247883356
--1.73946070989
--1.15404069458
--0.676413392183
--1.97720566122
--1.60580460299
--2.47941279308
--2.64450594643
--1.14325500958
--0.883452070899
--1.02715835527
--2.39120060203
--0.286444585964
--0.292468929367
--2.96955204468
--0.640641461839
--1.89970618863
--2.0446408882
--0.710884661776
--1.84851913914
--1.27465807562
--1.76718305202
--1.42603930516
--1.27392938671
--1.80694403787
--2.49172070939
--1.36854085973
--1.64625770966
--1.18431249106
--2.35737683119
--1.25001275682
--2.56322365712
--1.05185134491
--1.92127983023
--2.29447460927
--2.08890805134
--1.63522117309
--2.13736731335
--0.500480466614
--2.93612690939
--3.243496307
--0.69716412435
--1.20133591981
--0.192247316544
--2.63882773609
--1.17260729571
--1.27157218555
--2.78625736023
--1.40965083026
--1.03259007187
--1.61002340485
--2.22909812485
--1.28626095627
--0.645075126567
--1.29247284293
--2.51871134535
--0.496977446214
--3.1283912272
--1.44945963651
--2.7870761054
--1.507771836
--1.12586777736
--1.18037184781
--1.70521430688
--2.0659404202
--1.08907771288
--2.50545653904
--3.12646587617
--2.55620508861
--1.5001233841
--0.689497086859
--1.55374873054
--1.39637483567
--1.6707089582
--1.3738188529
--1.36808453714
--1.44613494995
--0.675560573629
--1.68192481042
--1.9456533601
--2.50696229173
--0.899859036565
--1.17189219914
--2.2392125728
--2.50406239469
-7.44737777701
-7.11545042049
-7.41511985868
-7.98566266781
-8.28214251943
-7.03152248834
-8.49796903192
-8.63022587271
-7.04783497069
-8.09631369837
-8.05288699837
-7.39529560261
-8.32423662326
-7.61822884086
-7.27602118028
-8.47639481778
-9.67524581114
-7.10619472148
-7.32993859186
-7.21426079956
-7.33316459683
-7.68501805198
-7.61588445378
-8.16851898185
-7.95082691052
-8.14625617137
-7.69293601922
-7.44399953435
-8.08924277926
-7.00522743918
-5.85476210549
-6.88213963966
-7.42436070747
-7.97678410835
-8.05917377536
-8.23917508924
-6.70753112697
-8.27078093127
-8.25612053769
-6.63376363543
-6.7469959036
-7.05164084636
-7.51107464785
-8.62687968831
-8.49378437151
-7.06447907188
-7.17942808703
-6.84062949086
-7.50366298294
-7.05236525662
-7.5336282563
-8.1776203378
-7.73498023446
-7.63116536811
-7.5681071008
-7.27056462566
-7.65159571234
-7.39053027433
-8.46283886034
-7.75642526507
-8.97851003445
-7.60997719986
-7.59245545732
-8.8089988415
-7.13962564712
-7.11651984533
-7.46963935054
-7.05947427095
-7.2212473968
-7.58895444334
-7.70303123668
-8.26045321407
-7.31615425437
-7.76675539286
-7.77112825339
-7.51483166984
-8.00729950163
-7.4569020033
-7.25495814564
-7.92491799517
-7.38467856269
-7.04381075604
-8.26654123761
-8.28796945506
-7.73636848258
-8.46386560703
-7.77289927677
-7.47189955211
-8.81349384373
-7.58313224823
-6.32779651679
-7.61326301593
-8.0055513573
-7.39375666268
-8.33133288953
-8.57361313699
-7.87655356786
-7.17770801287
-6.60899282768
-6.91105882231
-6.83627520619
-8.19902345855
-8.24368151175
-7.71138594367
-7.58352981298
-7.73430927512
-7.35986603721
-8.929172025
-7.72045048342
-7.77376640699
-8.63982449664
-7.57289787856
-6.98937112267
-7.8382832199
-7.95371487259
-9.09455133821
-7.41470299836
-7.63135475586
-7.60057454205
-8.07434921639
-7.84963296491
-7.94373314067
-6.9145246913
-7.53978835262
-7.6299739087
-7.55728319272
-7.85576927972
-7.63617888844
-8.47807963332
-8.26891797425
-6.38167041444
-7.12244637917
-8.0851752867
-6.80994878451
-7.13375361441
-8.88153757513
-7.93015424601
-7.91373598431
-9.2379141024
-7.67283224206
-7.82210466147
-7.55858554098
-8.94392451065
-7.6333468552
-7.34460074337
-7.5374378123
-9.05508584046
-8.7358231912
-7.0633666958
-7.67086280263
-7.61644191998
-8.6338624321
-8.06489376159
-8.78593812973
-7.54511999326
-6.24119281174
-8.10288052656
-8.34666116438
-7.31132651138
-8.50939310292
-6.24709749563
-8.27459460247
-7.58776206214
-6.91305991887
-8.12364111478
-6.43398216426
-8.13622891504
-8.22555980484
-7.48959542696
-7.60781613067
-7.99452324529
-7.81481411839
-8.18522687041
-8.46387971328
-9.34911491512
-6.59804070958
-7.41011774726
-7.97087988865
-8.34891763891
-7.86690431509
-7.82763085571
-8.74753787704
-8.07414266615
-8.32352699757
-8.78239572545
-8.89387188097
-8.40461859927
-7.63410535818
-7.09116414484
-6.82685640992
-7.95638773933
-7.49103865874
-7.34976070725
-7.59139028985
-7.42853609666
-8.99288722835
-7.32856743398
-7.1617136321
-8.80162919673
-8.32768905755
-7.20015620509
-7.00946884564
-7.5818706451
-8.6716183984
-7.16054269287
-7.86181888342
-7.22695321238
-8.00499593004
-7.52449060158
-8.12960555302
-6.48840875999
-7.03892492523
-7.07721791884
-7.67983258637
-8.26890513952
-7.45295083239
-7.87277395106
-7.34328153833
-8.10503002284
-7.91113159817
-7.25106933897
-8.88880044885
-7.61754341806
-7.89546362569
-7.58844865581
-7.64192093915
-7.12558085809
-8.30587371387
-6.87668681813
-7.39320662032
-6.49961554505
-7.81174176549
-8.37684114062
-8.09744197555
-6.60299974123
-7.18235233713
-6.93964358894
-8.6586786494
-7.85365011418
-7.09044258093
-7.96811194783
-7.6943098611
-7.35778292522
-7.4147934875
-8.03624814136
-6.72552980103
-8.0440951705
-7.64998153123
-7.21934543903
-7.00677298191
--1.01155659856
--0.0177884285435
-1.84000256954
--0.0204963456032
-0.969336008302
-1.23805624512
--0.580597910665
-0.979886657
-1.2441206062
-1.49167239728
-2.48498647727
-0.112215663671
-1.10475975165
-1.07401581929
-1.47639458
-1.12250835545
-1.76373034455
-2.2038861826
-1.67559333672
-1.28895927095
-0.636587051942
-0.608056153179
-0.619692793929
-0.0704446477538
-2.82143280657
-1.59754556249
-1.81932401395
-1.93848345969
-0.409246731059
-1.29592389641
-2.29856280153
-0.536880320905
-1.33895982834
-1.66511108323
-0.546574264441
--0.796094499666
-1.35668351765
-0.953194042146
-1.10154821942
-1.05636173151
-2.1586981489
-1.72787314252
-2.05470493655
-1.58094166619
-0.743700758022
-0.0235550230396
-0.179575696644
-0.961990969747
-2.69238859965
-1.67000820665
-1.49193388306
-2.21598147308
-0.95760506444
-2.71785949258
--0.145491559807
-1.17244568771
-0.651968872097
-1.83889459369
-1.29580312795
-3.86057789343
-2.38550094926
-0.741641931206
-2.13469023571
-2.07804076834
-1.86137643019
-1.25279967758
-2.27985270546
-2.17975067425
--0.325266685151
-1.35449547165
-3.13394508059
-1.56332723871
-2.29652863482
-1.71339503367
-2.52833654967
-0.0755184294676
-2.04617167631
-0.837293312916
-0.258258306882
-1.65647186841
-1.56271823663
-2.52368754745
-2.59234836775
-0.589023864185
--0.0777552028649
-1.55480007671
--1.47738184275
--0.409710011372
-0.322667763969
--1.17458848747
-2.81599150425
-2.57968195216
-1.1879775489
-1.09564704653
--0.215594158694
-1.10198344072
--0.472560439394
-3.53340086281
-0.723315648076
-1.63550247128
-1.05102801734
-2.21477023189
-0.947636769313
--0.500633915245
-1.37918683465
-3.67402736367
--0.107089292547
-0.0838619525856
-1.98250830477
-0.0123231042826
-1.00764470875
-0.865602802277
--0.0542918676424
-0.627185697026
-2.43622732485
-3.87295704182
-2.09380807811
-0.95352349076
-0.894625574084
-2.22266681917
-1.01197754252
-0.629657248505
-1.4727456739
-1.42636172735
-2.59374239066
-0.723895009699
-0.901713085509
-0.974496960427
--0.56575228702
-1.46420250916
-2.03071150824
-0.829880203838
-1.00736623279
-1.79717258154
-2.05228947921
-0.91429950604
-1.45866967879
-0.372524567582
-1.77805947561
-1.86887773848
-0.827370143185
-0.990330514014
-2.33207789743
-1.70868945119
-2.28408130833
-1.35577112727
-0.616830019348
-2.10733980015
-2.29545916244
-0.861302826076
--0.479357944483
-1.20185946888
-1.26749267695
-1.53963781702
-2.10475673373
-1.47087123585
-0.642158559413
-1.50390269013
-1.86533977056
--0.598012629533
-2.30795938753
-2.87470791916
-1.44421293362
-0.303594748139
-1.71471371538
--0.250791080313
--0.822378936886
-0.468601836098
-0.456335003714
-2.60399532133
-0.398790305089
--0.235951161057
-1.77991104731
-0.993421366509
-1.17940074872
--0.497926362074
-1.73252949475
-0.155570947222
-1.09351713245
-0.377026335699
-2.03056918481
-3.05212233931
-1.94899286655
--0.0347264876383
-2.54359007293
-1.23044110649
-1.2176593476
--0.193703716351
-1.17105522403
-0.284417290802
-1.09207214235
-2.38501290425
-3.3138431787
-1.73564244332
-2.04081606823
-1.58548837241
-1.33687906721
-0.680078563786
-0.748803536754
-1.61707985559
-1.37098558983
-2.38342783453
-1.20477600065
-1.69874045665
-1.40200450642
-1.54432431503
-1.66763183183
-0.942414942408
-2.33810140449
--0.923821630029
-1.9038605985
-1.35194738452
-0.534360234468
-1.49673798865
-1.10319670265
-1.26051347403
-1.50304357568
-1.0965268985
-1.87225120879
-1.9148780146
-1.37476868188
-1.83471772825
--0.952108633115
-0.812988951412
-1.96614716283
-0.559841161908
-1.18849154393
--1.22145567442
-3.23117223538
-1.39238890939
-1.39235933806
-3.85661815802
-1.02202948112
-2.94737899747
-1.48287969956
-0.753462161929
-2.39971180633
-2.15803641979
-1.45372673432
-0.56243524725
-0.248408548074
-0.484179465364
--0.652155177229
-0.399351354516
-3.11562115929
-0.997408835925
-2.77720738414
-1.1772772084
-0.589843212528
-1.25623085053
-8.06100159945
-8.29818809546
-9.30808607209
-8.68219341608
-8.77613108041
-8.44368872624
-8.10443804238
-9.32695302884
-8.32335549199
-9.06650262805
-8.6980286141
-8.72201971117
-8.79202969291
-9.35204900416
-8.6114209214
-9.23476751458
-8.35068864876
-8.94013011984
-9.06589245624
-8.04930316721
-8.85352725953
-9.30680941355
-8.67463207099
-8.53941736865
-9.17028197381
-8.5184992569
-9.43085846264
-8.4789966239
-9.38398511849
-8.9989419411
-8.74085241232
-8.91159341065
-9.12353471641
-8.67953061985
-8.74104707359
-8.34563898904
-8.26696073164
-8.50487466457
-8.81461180302
-9.09531308022
-8.55964128851
-8.38199964842
-8.54923307133
-8.23407295301
-8.7777016919
-9.19611663799
-8.89386392417
-8.76806239514
-8.40042991709
-8.10962983328
-8.92865328798
-8.53737534746
-9.48195723677
-8.67816145773
-9.18182264713
-9.01778572041
-8.74473602752
-8.50817337287
-7.78454562617
-8.51524032646
-8.62102102958
-9.29232460951
-9.06464006047
-8.50490507796
-8.61481438984
-8.33123904037
-8.43500112037
-9.04195291367
-8.49022294764
-8.353861265
-9.27676507328
-8.40353712277
-8.31520200665
-8.70833252165
-7.91914040291
-8.59073679319
-8.8770030124
-8.55123427037
-8.83704852025
-8.50719551073
-9.16143366782
-8.98095272512
-9.35585811363
-7.88252617272
-8.99991391181
-9.34861697607
-9.49441157892
-8.52815527514
-9.06562583682
-8.94786239534
-8.48405231641
-8.52397498938
-8.30417899651
-8.77831764058
-9.15014890241
-9.36814412751
-8.4786317279
-8.73756168307
-8.49366372473
-9.26195852806
-8.84327711697
-8.5809146606
-8.58308783269
-8.93479887043
-8.9585557649
-8.81768461409
-9.0793983634
-9.01887992702
-8.64935545407
-8.74987846444
-9.45214992729
-8.5237984374
-8.44049840677
-7.75909711143
-9.02864882044
-8.90017687324
-9.02386728954
-8.19444858827
-8.66247131206
-8.41070451367
-9.05254198226
-8.70889217506
-8.59354925442
-8.48651426978
-8.64414128373
-8.56671619887
-8.57186120675
-9.10624758185
-8.70605784795
-8.82575798428
-9.1184640408
-8.64285689904
-8.64143704546
-8.41876427484
-9.14384562708
-8.41662377722
-9.24105609243
-8.66804278373
-9.32116283067
-9.45748775015
-8.96259760519
-9.27971194193
-8.96106705323
-8.12456116974
-8.06905633333
-7.80566592407
-9.10774185025
-8.32329040395
-8.95280107742
-8.57851594888
-9.54967585701
-9.12197761731
-8.2473609637
-8.5835972551
-8.60945660178
-8.5018707731
-8.38586090262
-8.70064243717
-8.02633837358
-9.01362446175
-7.93787405869
-8.78150855656
-8.69680943789
-8.69531594776
-0.0
-9.09233369123
-8.66596335904
-8.25428440083
-8.83021915243
-9.11632149789
-9.28581175306
-7.91022935266
-8.98040477914
-8.8416473723
-8.85958926526
-8.88138370281
-8.67358087743
-8.89514783651
-9.26017531229
-8.81063465933
-8.75906654446
-9.14970356703
-8.36131802604
-8.81061459442
-8.38095917498
-9.15612092914
-8.50320007504
-9.0584853729
-8.48075480537
-8.38666310462
-9.03072396209
-8.93239783837
-9.42062823617
-8.36847687272
-9.333986456
-8.12261583461
-8.77450417854
-8.39610418458
-8.93943759314
-8.74963645356
-8.83349495172
-8.50383053798
-8.6946406297
-8.61464333276
-8.81699765443
-8.67161924619
-8.19732517878
-8.62366986272
-8.25263991753
-8.72620740042
-8.51387385344
-8.49799975235
-9.07550993182
-8.433605446
-7.82339187365
-8.40991600493
-8.14783558484
-8.48192190124
-8.88415085035
-9.1361773063
-8.93349324117
-9.03938266747
-8.85737843918
-8.99553304422
-8.50654702485
-8.5918397594
-8.97849798562
-8.72593494475
-8.98900087787
-8.8790453009
-8.42379291586
-8.94846248916
-9.14984614209
-8.70324041002
-8.85143197087
-9.09731829707
-9.34901304478
-8.18611118721
-8.22878782434
-8.78324694115
-9.49247495601
-9.02337640697
-9.41098248566
-8.90597322816
-8.72681189013
-8.28822431623
-8.45438278261
-8.5571674186
-9.19970800383
-9.23966819862
-9.18066360078
-1.17627891497
-1.50306863789
-1.05888748319
-1.11427138302
-1.2804869857
-1.37824926248
-1.35306165008
-1.5307415116
-1.39939046306
-1.15336970828
-1.325030119
-1.22996825786
-1.44153029818
-1.01249759626
-1.25048231195
-1.4365922365
-1.31786800873
-1.20885094621
-1.16043848184
-1.06841222899
-1.22635026886
-1.18131144343
-1.21495592238
-1.29823619286
-1.26647159517
-1.30793307015
-1.2347079396
-1.45660485538
-1.13474942768
-1.41689643244
-1.38528508915
-1.37301478685
-1.11448797365
-1.30069959827
-1.42719411173
-1.30453192748
-1.07951299542
-1.21305232438
-1.34332249457
-1.19206410147
-1.00870716969
-0.910992907734
-1.54064329606
-1.31890459531
-1.25250620415
-1.25248184831
-1.28027487331
-1.11782105806
-1.19347317189
-1.49777153336
-1.19320886098
-1.27960848712
-1.18613550129
-0.957293265068
-1.25707759022
-1.094740193
-1.28440418792
-1.04302262239
-1.2340591624
-1.43167748313
-1.13292258769
-1.24766418157
-0.916894802205
-1.22894466475
-1.22724508379
-1.12847652551
-1.08268663153
-1.30067674054
-1.2219240948
-1.37422600173
-0.957624013243
-1.19232555134
-1.1129040434
-1.44211992498
-1.19149401605
-0.983828003738
-1.25839986993
-1.08973615919
-1.02790927104
-1.29996147913
-1.10605601442
-1.22853397734
-1.28727959539
-1.51017594437
-1.29708621748
-1.21239690345
-0.967749363437
-1.38221913184
-0.919216578601
-1.60083004138
-1.35527911747
-1.0338617693
-1.1209924482
-1.39017987196
-0.965647733806
-1.15249167069
-1.058037471
-1.06425748624
-0.707314264548
-1.41747782029
-1.5247933961
-1.18813623489
-1.48691061559
-1.18001123462
-1.23228295892
-1.23654623885
-1.14483643208
-1.32164481647
-1.51865274089
-1.2131039312
-1.35531835658
-1.49265612026
-1.07019896531
-1.25883398864
-1.07824521658
-1.13067196103
-0.978827745894
-1.41521597346
-1.19336060875
-1.27001869741
-1.2505904824
-1.2828055869
-1.01103981442
-1.25615189671
-1.36167286104
-1.33709344183
-1.18297748418
-1.19435637019
-1.5353833486
-1.33599066817
-1.25327296578
-1.08357149082
-1.35855366959
-1.05964685977
-1.27110127186
-1.12977175742
-1.07739539825
-1.45174555064
-1.17862570746
-0.949313244504
-1.62493048718
-1.27229075588
-1.0881435772
-1.11294400585
-1.27306510303
-1.37060504984
-1.30336317544
-1.07036133718
-1.37988465661
-1.0258645398
-1.05752568184
-1.54170305914
-1.1144104547
-1.07422579046
-1.09690015124
-1.26217903574
-1.31447064003
-1.33361194262
-1.24048955871
-1.19706053679
-1.07592336038
-1.2477621175
-1.07411340376
-1.49770028732
-0.979041602802
-1.1247768349
-1.06154308838
-1.24311094234
-1.4036879614
-1.32006981717
-1.07610557539
-1.40493729964
-1.34423343649
-1.21352325827
-1.39448570694
-1.24408805977
-1.18458394881
-1.04233688147
-0.954625186791
-1.21817895579
-1.21033169825
-0.999077136861
-0.851986648739
-1.09556885414
-1.28283642303
-0.648433740277
-1.15323616432
-1.10899176914
-1.61441932938
-1.03107310963
-1.13086027879
-1.11010060713
-1.20270283848
-1.17514433994
-1.19262016758
-1.16413608735
-1.29263695484
-1.21213716063
-1.00912927971
-1.36099668765
-1.31565778778
-1.14796882785
-1.01081488561
-1.38486457537
-1.38716591773
-1.16766128053
-1.24524531304
-1.48721412599
-1.15519982692
-1.12862467179
-1.37816694511
-1.06258831587
-1.29560274198
-1.04246334135
-1.3541481927
-1.51676114539
-1.3990383795
-1.30750677301
-1.53576897346
-1.19900470072
-1.11407739652
-1.09114097348
-1.02565662442
-1.06147613991
-1.25862035461
-1.21245421753
-1.03238551344
-1.1395260604
-1.11986235412
-1.47514449545
-1.13442025774
-1.27705148834
-1.16291220439
-1.25094755706
-0.934987830562
-1.34854764305
-1.25887673227
-1.03810941143
-1.24921844977
-1.3315136255
-1.31286821103
-0.98235518465
-1.5484282567
-1.38431355834
-1.24145998051
-1.40986614401
-1.33209143764
-1.11012065582
-1.25990372105
-1.32855089524

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/test/resources/hivemall/anomaly/cf1d.csv.gz
----------------------------------------------------------------------
diff --git a/core/src/test/resources/hivemall/anomaly/cf1d.csv.gz b/core/src/test/resources/hivemall/anomaly/cf1d.csv.gz
new file mode 100644
index 0000000..be93f41
Binary files /dev/null and b/core/src/test/resources/hivemall/anomaly/cf1d.csv.gz differ

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/test/resources/hivemall/classifier/news20-small.binary.gz
----------------------------------------------------------------------
diff --git a/core/src/test/resources/hivemall/classifier/news20-small.binary.gz b/core/src/test/resources/hivemall/classifier/news20-small.binary.gz
new file mode 100644
index 0000000..19a0199
Binary files /dev/null and b/core/src/test/resources/hivemall/classifier/news20-small.binary.gz differ

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/test/resources/hivemall/fm/5107786.txt
----------------------------------------------------------------------
diff --git a/core/src/test/resources/hivemall/fm/5107786.txt b/core/src/test/resources/hivemall/fm/5107786.txt
deleted file mode 100644
index a9f7170..0000000
--- a/core/src/test/resources/hivemall/fm/5107786.txt
+++ /dev/null
@@ -1,200 +0,0 @@
-1.0 15998738 718187 6597286 593672 1234506 6642310 3780405 16373656 5151837 
-1.0 4833863 3306359 6597286 12335786 
-1.0 13273702 8793953 6597286 1234506 6642310 3780405 5151837 
-5.0 6904965 387280 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 1059368 12938118 9300604 
-2.0 4083424 16604718 16373656 
-1.0 12671616 3349410 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 8324914 12922968 7116522 
-2.0 3741433 2128137 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 855090 16367697 7116522 6597286 12335786 16454522 
-3.0 10849674 5586750 7116522 
-2.0 10716018 7195547 7116522 
-1.0 10799582 14376540 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 7285675 5193573 6597286 13411118 1234506 6642310 5151837 
-2.0 15768637 7626535 8051895 11975562 
-2.0 5186920 1392416 7116522 16454522 
-2.0 14543045 7262196 7116522 12335786 16454522 
-1.0 13964892 7171547 1234506 3780405 
-2.0 16067673 4031531 16454522 9300604 
-1.0 6887043 14170192 7116522 6597286 12335786 
-4.0 16127003 10561437 6710760 7116522 6597286 16373656 9381434 
-1.0 13246939 12124885 7116522 6597286 10485214 4018897 16454522 
-1.0 15874619 7216361 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-2.0 9473525 7624415 7116522 
-1.0 4713388 14835441 3780405 2134114 
-1.0 9959781 9281617 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 1509899 5750906 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 13998309 387280 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-3.0 7570662 4996734 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-5.0 13993342 13237699 8932716 
-1.0 4301397 10438333 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 12518207 2161920 16373656 
-1.0 8851260 6893006 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 2434514 15870546 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 13499900 1818177 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-3.0 6279958 3932474 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 8711054 8459993 6597286 1234506 8051895 13435265 8932716 
-1.0 166880 1296179 7116522 
-5.0 2618419 14575149 7116522 
-1.0 6809574 3539528 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 11540890 8769952 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-3.0 13034153 7874204 1234506 6642310 3780405 5151837 
-1.0 16452117 3595708 8051895 3780405 5151837 8932716 11975562 
-2.0 5055738 12395440 16454522 
-1.0 14782360 8385822 7952522 3711888 16373656 5151837 8932716 
-1.0 6943898 528242 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 10573081 5439570 7116522 
-5.0 10329498 3346400 12335786 3711888 
-1.0 13000708 13348770 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 7461009 1590113 8932716 
-1.0 5702914 5709158 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 1546172 7709187 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 9480751 6017034 6597286 10485214 593672 1234506 6642310 3780405 5151837 
-1.0 9931715 4802118 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 2173252 7595613 6597286 16454522 3780405 8932716 
-1.0 6125592 9407384 9381434 
-1.0 14864131 9507940 7116522 16454522 
-1.0 4466303 8529724 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-2.0 8780864 425514 16454522 2134114 8932716 
-1.0 3378282 12162667 7116522 8051895 
-5.0 524716 812412 7116522 
-1.0 12151684 937201 9381434 
-1.0 5609570 10810596 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-2.0 15829774 6837842 13435265 
-3.0 10868053 7262196 7116522 12335786 16454522 
-1.0 12462417 12178393 7116522 
-1.0 9071291 1437382 12335786 3711888 
-1.0 9825496 10646267 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 4485167 9668403 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 2688816 11308264 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 12143742 5650995 9300604 
-2.0 8260054 10453091 9300604 
-1.0 6174572 11504706 10485214 3711888 
-1.0 7864442 16604718 16373656 
-2.0 5932172 3636319 16373656 
-1.0 8486030 90324 10485214 
-1.0 2256551 12291454 7116522 9300604 
-3.0 15191779 14473987 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 7992534 6979142 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 1193922 6060893 1234506 6642310 3780405 5151837 
-2.0 12544303 6392648 1234506 6642310 8051895 16373656 
-5.0 5484736 5710773 12335786 7952522 8051895 
-1.0 3327249 10159121 7116522 16454522 
-5.0 4142408 15998670 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-5.0 14754505 379922 6710760 7116522 6597286 12335786 4018897 
-1.0 8063338 3395251 6597286 6642310 8051895 3780405 16373656 5151837 9300604 
-1.0 3402955 3349410 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 16473159 15785240 1234506 6642310 13435265 3780405 2134114 5151837 
-3.0 6445758 13358419 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-3.0 10272898 844045 7116522 13411118 16454522 5151837 
-3.0 13302589 12159466 1234506 6642310 8051895 
-1.0 5979984 12160944 9381434 
-1.0 3595558 16198471 1234506 11975562 
-1.0 12546388 9682670 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 9434795 2050202 12335786 8932716 
-1.0 3548553 15061809 7116522 6597286 12335786 16454522 
-1.0 15447934 7530429 9381434 
-1.0 4924920 11501943 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-5.0 8174682 3333357 12335786 13411118 6642310 8051895 16373656 
-1.0 9152776 7447730 16373656 
-1.0 12704039 14702193 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-2.0 2173252 13701259 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 3855974 8527607 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 6141008 11308264 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 107852 15403273 9381434 
-1.0 13502554 10643244 7116522 
-1.0 10679337 2868706 7116522 12335786 10485214 16454522 
-1.0 4678895 15415020 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-2.0 7679834 14881428 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 113243 8629496 7116522 6597286 8051895 16373656 
-1.0 5480499 1920930 7116522 6597286 16454522 
-1.0 4310332 2603068 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-5.0 7987608 16725533 7116522 6597286 
-1.0 3094558 1850018 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 13186758 1392416 7116522 16454522 
-1.0 4805879 13112577 9381434 
-2.0 12348903 10218316 4018897 11975562 
-1.0 15672026 12774486 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-4.0 6725952 13757560 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-2.0 1786203 16107700 6710760 13411118 
-1.0 9472119 9055785 7116522 6597286 
-1.0 5049306 5586750 7116522 
-1.0 604766 5800288 13435265 
-5.0 13014214 14850018 6597286 12335786 8051895 3711888 5151837 
-1.0 11558204 7506665 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 3377643 2147993 6710760 7116522 6597286 1234506 13435265 
-1.0 7047541 7396683 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 9744209 10483233 7116522 6597286 8932716 
-1.0 1185304 466985 8051895 13435265 8932716 
-1.0 10097296 5855826 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 633757 14323246 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 16093534 15149568 7116522 6597286 16454522 
-2.0 6046738 2990274 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 9424532 12292316 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 12661007 12670472 16454522 9300604 
-1.0 40477 782023 7116522 16373656 9300604 
-1.0 13690297 13304874 7116522 12335786 16373656 
-1.0 12000297 16012501 6710760 
-1.0 10736409 6743491 9381434 
-1.0 11214727 2445260 8932716 
-1.0 5748936 7037018 10485214 
-1.0 14748801 1238701 6597286 10485214 4018897 13435265 
-2.0 4654213 2277251 9300604 
-2.0 2442013 6342696 8932716 
-3.0 6076879 8033059 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-2.0 10505995 528242 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-2.0 13036044 7243673 6642310 3780405 
-2.0 4028177 322234 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 4333354 4314451 12335786 
-2.0 16267524 7592680 9300604 
-5.0 7435936 8527607 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 1529779 5466206 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 7434783 85493 1234506 6642310 16373656 5151837 
-4.0 13591789 2128137 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 1295499 14940332 6597286 8051895 13435265 8932716 
-1.0 7398687 6387824 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-2.0 871149 15387560 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-2.0 7727111 1637370 6710760 6642310 5151837 
-5.0 4769362 3993383 7116522 6597286 16454522 
-1.0 483039 7776221 1234506 3780405 
-1.0 3731335 9119321 1234506 3780405 5151837 
-2.0 2810525 13876811 9300604 
-5.0 9519257 6698889 9300604 
-3.0 6440693 15998670 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-3.0 10895286 15789153 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 4485167 11716956 12335786 9381434 
-1.0 4781765 10911433 7116522 6642310 8051895 5151837 
-2.0 12868358 9508152 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 11042882 13678061 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-2.0 10028988 14173235 7116522 6597286 16454522 3780405 9300604 
-1.0 4706190 4441642 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-3.0 11391417 2825661 16454522 
-1.0 9230576 5596525 12335786 
-1.0 5207593 7143541 6710760 7116522 6597286 9381434 
-1.0 9353422 12712891 16454522 
-1.0 1036574 3954723 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-2.0 14921191 5066114 3305095 9381434 
-5.0 1816032 6611865 7116522 6597286 
-3.0 6077009 10315434 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 15886997 2326501 6597286 6642310 5151837 
-1.0 11697271 2149806 13411118 10485214 4018897 7952522 5151837 
-1.0 8494551 8645188 1234506 13435265 3780405 
-4.0 10139879 1374403 7116522 9300604 
-2.0 16757684 2277251 9300604 
-1.0 4302963 12293902 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-3.0 3153648 8033059 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-5.0 14389782 6090248 8932716 9300604 
-2.0 14250513 16604718 16373656 
-2.0 1315141 16654550 13435265 16373656 8932716 
-1.0 12674036 5294254 6597286 593672 1234506 6642310 13435265 3780405 16373656 5151837 
-1.0 12651283 6023644 6597286 8051895 
-2.0 886695 662655 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 2764815 2559103 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-2.0 15392891 16604718 16373656 
-5.0 3296092 6957898 10485214 7952522 
-1.0 9989248 16385707 16373656 
-1.0 563138 7837109 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 
-1.0 8853714 11822042 7116522 6597286 16454522 6642310 5151837 8932716 
-1.0 15816411 8443010 6597286 1234506 6642310 5151837 
-1.0 1156210 5120672 6710760 7116522 6597286 12335786 13411118 10485214 4018897 7952522 16454522 593672 1234506 6642310 8051895 3711888 13435265 3780405 2134114 16373656 3305095 5151837 8932716 4202136 9381434 11975562 9300604 

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/test/resources/hivemall/fm/5107786.txt.gz
----------------------------------------------------------------------
diff --git a/core/src/test/resources/hivemall/fm/5107786.txt.gz b/core/src/test/resources/hivemall/fm/5107786.txt.gz
new file mode 100644
index 0000000..1b036cb
Binary files /dev/null and b/core/src/test/resources/hivemall/fm/5107786.txt.gz differ


[2/6] incubator-hivemall git commit: Close #13: Implement Kernel Expansion Passive Aggressive Classification

Posted by my...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/test/resources/hivemall/mf/ml1k.train
----------------------------------------------------------------------
diff --git a/core/src/test/resources/hivemall/mf/ml1k.train b/core/src/test/resources/hivemall/mf/ml1k.train
deleted file mode 100644
index 11c6177..0000000
--- a/core/src/test/resources/hivemall/mf/ml1k.train
+++ /dev/null
@@ -1,80000 +0,0 @@
-533 430 5
-450 182 5
-428 690 5
-279 1361 3
-180 433 5
-409 430 4
-327 66 3
-303 847 4
-896 597 4
-474 13 5
-62 448 2
-348 118 4
-390 283 4
-497 402 4
-328 657 4
-17 117 3
-805 558 5
-35 358 1
-635 15 3
-690 211 3
-682 765 4
-1 33 4
-833 135 4
-145 227 4
-79 276 3
-864 1101 4
-92 204 4
-807 449 5
-712 220 5
-747 318 5
-90 60 4
-349 276 5
-891 116 3
-943 722 3
-323 282 3
-276 682 3
-398 430 4
-939 257 5
-85 121 2
-100 751 4
-311 654 3
-263 323 1
-735 50 5
-212 199 5
-804 2 4
-409 433 4
-1 170 5
-58 182 4
-846 1267 3
-296 654 5
-425 259 1
-437 239 4
-727 205 5
-135 294 4
-870 1014 2
-455 117 3
-893 819 3
-454 751 4
-276 373 2
-514 216 5
-491 7 3
-809 302 5
-450 252 3
-537 65 3
-450 234 3
-123 132 3
-477 25 5
-28 50 4
-172 772 1
-233 483 5
-753 242 4
-880 182 5
-796 432 2
-104 871 2
-102 248 3
-318 865 2
-270 509 3
-640 269 5
-127 229 5
-878 553 3
-931 1152 4
-22 265 3
-796 161 5
-537 558 4
-897 118 5
-201 1423 3
-267 597 3
-807 422 4
-927 95 5
-756 731 3
-484 82 4
-373 143 3
-318 1050 4
-215 692 3
-380 217 2
-458 32 4
-363 472 1
-394 554 4
-572 813 4
-807 199 5
-773 710 3
-238 237 3
-724 331 3
-807 210 4
-479 340 1
-279 401 5
-500 1014 2
-661 194 5
-15 845 2
-1 186 4
-181 870 2
-896 248 4
-249 741 4
-230 202 4
-234 989 2
-59 462 5
-369 196 5
-232 483 5
-447 121 5
-102 384 2
-776 486 4
-303 692 4
-901 378 5
-91 327 4
-215 1063 5
-315 183 3
-888 69 4
-622 1408 1
-682 96 4
-328 350 3
-751 472 2
-561 510 3
-764 819 3
-711 286 4
-158 516 5
-104 456 3
-472 685 3
-293 81 4
-178 316 4
-354 529 4
-690 98 5
-342 251 5
-931 546 3
-679 215 3
-298 69 4
-92 120 2
-787 899 3
-673 242 4
-743 748 4
-24 41 5
-130 79 5
-280 586 4
-505 54 3
-226 169 5
-908 173 3
-606 3 5
-234 30 4
-877 955 4
-303 545 2
-758 387 2
-116 187 5
-301 673 4
-449 170 4
-812 261 1
-455 286 5
-923 245 3
-125 90 5
-429 12 5
-546 379 4
-256 117 5
-334 425 4
-274 277 4
-914 111 1
-450 328 4
-345 137 4
-487 470 5
-875 134 5
-389 610 5
-314 845 5
-343 286 4
-534 1052 4
-379 97 3
-878 215 2
-484 56 5
-608 182 4
-85 1170 3
-887 559 4
-243 86 5
-398 427 4
-861 1009 5
-509 289 2
-206 260 3
-64 1065 1
-655 344 4
-916 202 3
-279 625 3
-752 748 4
-83 929 3
-916 213 4
-586 581 2
-269 763 1
-327 89 4
-446 327 2
-289 926 3
-400 748 2
-637 15 4
-234 1446 3
-807 69 5
-82 13 2
-7 588 4
-26 760 1
-509 343 3
-830 692 4
-538 22 5
-741 226 2
-94 154 5
-490 126 2
-75 756 2
-648 295 4
-749 284 4
-344 546 3
-162 230 2
-354 180 3
-774 273 1
-749 79 4
-208 996 3
-500 223 4
-60 528 4
-550 258 5
-151 89 5
-848 165 5
-942 498 5
-455 515 4
-126 303 3
-918 507 5
-692 168 2
-150 458 4
-503 435 3
-263 271 1
-705 546 3
-459 934 3
-194 168 5
-109 317 2
-478 467 5
-189 517 4
-807 102 4
-335 678 3
-279 208 5
-26 742 3
-367 379 4
-297 196 4
-593 280 3
-391 661 5
-643 50 4
-325 134 4
-653 563 1
-299 88 3
-840 212 4
-224 518 1
-846 731 3
-160 117 4
-125 364 3
-724 264 3
-843 650 3
-577 191 4
-875 479 4
-425 327 4
-676 508 1
-487 197 3
-405 401 1
-666 213 4
-109 405 5
-919 953 3
-480 234 4
-588 382 3
-435 476 3
-75 237 2
-13 549 4
-587 342 1
-936 301 3
-303 577 3
-405 920 1
-393 274 4
-865 597 1
-387 508 4
-541 543 4
-137 96 5
-655 281 2
-843 164 3
-496 141 3
-516 286 5
-544 323 2
-313 497 4
-435 577 3
-899 748 4
-326 227 3
-713 310 4
-329 8 2
-655 515 4
-758 752 3
-608 479 5
-650 265 4
-54 871 5
-22 862 1
-110 94 4
-75 410 5
-417 131 4
-303 387 5
-216 402 2
-234 208 4
-936 300 3
-881 194 3
-269 629 2
-59 610 4
-840 430 5
-122 708 5
-224 403 4
-423 750 5
-740 289 4
-452 654 2
-853 333 4
-823 1118 3
-858 754 4
-441 121 4
-393 204 4
-889 129 5
-174 332 5
-184 371 5
-750 294 4
-366 200 5
-601 118 1
-435 890 1
-311 101 4
-788 693 2
-540 1011 4
-216 151 3
-270 93 5
-727 197 3
-254 141 3
-527 203 4
-911 501 3
-216 943 5
-134 294 4
-360 100 5
-155 748 2
-734 282 4
-429 999 2
-212 511 4
-429 568 3
-292 193 4
-109 409 2
-535 64 5
-328 474 4
-409 514 5
-450 418 4
-189 498 5
-882 33 2
-299 274 3
-815 443 3
-416 269 4
-308 74 4
-766 523 3
-308 660 3
-479 1028 1
-845 1234 4
-766 465 3
-822 101 2
-742 124 4
-125 1037 2
-568 529 4
-87 546 3
-642 140 3
-626 268 4
-846 514 3
-234 504 4
-485 345 1
-942 1028 4
-715 248 4
-932 679 2
-286 537 4
-839 255 3
-704 318 5
-830 313 5
-337 230 5
-280 22 5
-84 1047 2
-464 358 3
-378 720 2
-543 936 4
-178 508 3
-474 735 4
-617 217 1
-392 310 4
-399 969 3
-228 56 2
-301 684 3
-105 271 2
-625 479 4
-330 501 5
-697 683 1
-262 660 4
-505 154 1
-532 332 4
-59 608 4
-525 252 3
-521 755 3
-380 382 3
-537 922 3
-332 300 5
-291 686 5
-807 515 4
-722 628 4
-814 665 4
-588 1180 2
-927 230 5
-942 261 4
-823 53 5
-246 132 4
-152 685 5
-452 455 1
-697 307 4
-21 259 2
-40 321 4
-498 134 3
-409 1194 5
-916 527 4
-908 50 4
-450 714 4
-301 218 4
-221 1016 3
-552 50 4
-658 772 3
-498 887 3
-85 507 4
-374 223 5
-459 100 1
-916 72 3
-569 321 4
-573 427 4
-142 350 4
-12 381 4
-642 138 4
-848 204 5
-336 1037 1
-770 298 4
-622 809 2
-870 154 4
-723 289 2
-846 97 4
-800 237 4
-399 732 2
-672 931 1
-455 69 4
-487 597 4
-194 383 1
-269 194 5
-14 96 4
-381 771 2
-201 288 4
-326 64 4
-42 216 5
-279 490 3
-72 100 5
-727 184 3
-303 748 2
-802 330 2
-639 14 5
-18 197 4
-240 307 4
-385 219 1
-815 559 3
-411 568 4
-437 1211 4
-339 498 4
-471 432 1
-269 59 4
-189 194 5
-85 449 4
-504 1037 1
-854 133 3
-697 1047 3
-514 272 4
-554 294 3
-763 367 3
-885 549 3
-316 275 5
-332 405 4
-930 113 5
-863 886 3
-835 237 4
-655 1012 3
-852 260 3
-924 117 2
-49 369 1
-462 539 3
-535 505 4
-295 94 4
-929 318 4
-658 923 3
-125 401 4
-145 53 2
-393 934 3
-31 875 4
-943 38 3
-308 863 3
-796 615 4
-747 432 5
-933 452 1
-10 610 4
-588 542 3
-585 919 2
-291 179 5
-548 477 1
-854 195 3
-406 211 5
-184 411 3
-393 982 3
-783 334 3
-596 50 5
-525 596 4
-919 151 4
-141 823 3
-130 202 5
-622 166 5
-297 183 4
-349 412 1
-799 191 3
-308 655 4
-401 117 3
-435 273 5
-276 250 4
-590 130 1
-152 1136 5
-686 176 3
-560 1265 3
-630 845 3
-90 954 4
-747 423 5
-425 300 2
-267 552 3
-83 235 1
-201 1229 3
-16 168 4
-164 826 4
-631 886 4
-815 226 3
-782 538 4
-397 588 4
-450 71 3
-854 499 4
-535 591 4
-417 418 4
-846 1110 3
-332 769 3
-326 651 4
-201 751 3
-615 197 4
-327 1141 3
-82 147 3
-455 692 3
-527 69 4
-629 86 5
-555 265 3
-277 1008 3
-793 823 3
-421 238 5
-766 197 3
-188 210 4
-454 642 2
-474 212 4
-932 151 3
-158 239 3
-223 332 4
-758 364 4
-534 471 5
-621 804 4
-450 528 5
-73 271 2
-795 742 2
-825 411 3
-883 1462 5
-387 550 2
-724 880 3
-907 596 4
-798 112 3
-886 623 1
-361 66 4
-677 988 4
-492 137 4
-437 497 5
-773 924 1
-344 148 2
-635 323 3
-381 77 2
-16 657 5
-532 311 2
-625 121 3
-609 908 1
-85 639 3
-886 92 3
-405 132 5
-450 89 5
-802 669 1
-293 12 4
-705 560 2
-727 423 3
-536 52 3
-126 881 5
-59 505 4
-383 464 4
-458 25 1
-885 501 3
-645 135 5
-194 546 3
-650 132 4
-374 265 5
-101 1057 2
-347 829 4
-647 82 4
-189 56 5
-70 820 1
-545 1188 3
-757 232 3
-689 1 3
-236 423 5
-749 135 4
-747 288 4
-303 123 4
-379 193 4
-537 23 4
-15 934 4
-5 168 3
-798 707 2
-551 672 1
-114 100 5
-235 419 5
-686 179 5
-551 627 3
-120 508 2
-38 681 5
-423 10 4
-655 203 3
-905 742 4
-790 135 3
-658 98 4
-352 228 3
-721 872 3
-879 117 4
-279 802 4
-502 328 4
-645 674 3
-579 1047 3
-6 246 3
-293 33 2
-668 269 5
-339 76 3
-13 712 4
-279 751 4
-880 475 4
-145 592 3
-270 740 5
-142 82 4
-474 252 4
-796 249 1
-551 765 1
-130 1013 4
-862 260 5
-311 203 5
-416 1011 4
-207 18 2
-383 475 2
-655 605 3
-399 825 2
-293 559 2
-496 1063 3
-321 647 3
-178 127 5
-766 1050 3
-429 473 3
-84 258 4
-207 187 5
-849 121 5
-450 940 2
-249 1073 4
-42 660 3
-717 262 4
-577 50 4
-334 485 3
-290 476 3
-846 496 3
-719 64 5
-437 521 4
-271 257 4
-561 159 1
-886 229 3
-711 248 5
-274 496 5
-567 59 5
-684 1283 3
-263 79 4
-268 732 3
-712 173 5
-667 197 4
-286 1504 4
-181 748 1
-184 185 4
-99 28 3
-251 79 5
-686 22 5
-308 143 4
-462 288 5
-865 101 1
-880 250 3
-435 841 2
-893 122 2
-815 392 4
-95 257 5
-788 130 2
-263 181 4
-347 404 4
-554 68 2
-347 418 4
-334 1524 4
-158 978 3
-87 209 5
-739 661 2
-586 411 2
-16 151 5
-751 487 5
-488 485 3
-64 144 3
-470 248 3
-21 244 4
-374 1513 2
-760 375 4
-727 217 3
-343 473 3
-87 709 3
-62 1107 1
-60 178 5
-276 198 5
-450 315 4
-152 22 5
-296 475 4
-57 744 5
-720 310 4
-465 588 4
-193 443 4
-853 332 3
-815 258 4
-504 401 2
-34 899 5
-177 1110 3
-562 148 5
-468 137 4
-232 56 5
-43 950 3
-374 665 4
-181 1339 1
-336 401 1
-627 660 4
-847 301 5
-313 187 4
-311 504 4
-828 1196 2
-671 849 3
-773 64 4
-535 301 4
-13 225 2
-178 265 5
-398 8 3
-303 729 3
-109 755 5
-121 472 3
-209 258 2
-119 562 4
-648 928 4
-18 188 3
-336 368 1
-130 63 4
-91 134 4
-394 265 4
-159 195 3
-907 287 4
-84 12 5
-389 693 4
-608 332 4
-846 521 3
-279 269 4
-405 1545 2
-682 328 3
-429 469 4
-454 387 2
-101 1051 2
-454 589 2
-155 324 2
-151 724 4
-577 472 4
-6 87 4
-262 421 4
-276 1208 3
-385 325 4
-42 1047 4
-56 281 2
-284 906 3
-885 186 4
-710 286 4
-147 904 5
-887 22 5
-710 334 2
-682 181 5
-159 1190 5
-561 1110 2
-712 625 3
-783 288 3
-254 234 4
-868 151 5
-830 79 4
-749 729 4
-11 90 2
-13 895 1
-919 23 3
-115 279 3
-870 481 4
-919 16 4
-22 105 1
-943 282 5
-454 602 2
-244 255 2
-346 157 3
-727 435 3
-127 227 4
-728 100 5
-537 190 4
-719 9 4
-394 576 2
-712 386 3
-488 260 2
-922 216 3
-332 144 5
-823 625 4
-883 707 3
-655 410 2
-312 529 5
-907 15 5
-456 289 4
-160 412 3
-457 275 5
-659 58 4
-28 195 4
-370 238 4
-699 820 2
-591 211 4
-785 22 4
-164 121 5
-842 258 3
-622 217 4
-666 238 4
-677 268 5
-625 640 3
-176 324 5
-425 32 3
-867 1039 5
-144 516 2
-721 269 5
-942 705 4
-121 275 4
-267 135 5
-446 688 2
-326 183 5
-290 22 5
-326 528 3
-201 640 4
-749 941 5
-252 1 5
-892 110 3
-929 474 4
-899 200 4
-320 679 4
-931 344 4
-416 692 5
-922 294 4
-665 427 5
-747 959 5
-393 323 2
-450 1284 3
-63 302 3
-435 748 4
-776 191 5
-934 152 4
-119 392 4
-274 924 3
-13 749 3
-840 588 4
-413 15 4
-26 127 5
-747 47 5
-642 411 5
-600 1228 2
-299 1506 4
-8 79 4
-158 70 4
-393 1076 3
-838 289 5
-38 465 5
-378 382 4
-873 342 4
-591 66 2
-320 89 4
-806 196 5
-708 322 3
-932 671 3
-846 836 5
-385 145 1
-92 474 4
-378 956 3
-927 195 4
-349 596 2
-49 111 2
-897 1028 4
-405 62 1
-881 77 2
-821 1084 5
-308 216 3
-500 727 2
-246 223 5
-524 13 4
-18 224 5
-336 1183 1
-95 506 3
-540 300 3
-296 301 5
-5 416 1
-457 679 4
-318 629 4
-450 111 4
-429 28 3
-345 1016 3
-149 319 2
-825 687 5
-291 456 3
-42 357 5
-745 510 3
-217 233 4
-288 210 3
-639 971 4
-855 462 4
-119 222 5
-577 795 3
-642 815 4
-826 29 3
-883 736 3
-862 282 5
-896 191 4
-804 1188 2
-666 684 4
-617 446 2
-325 208 3
-588 181 5
-86 258 5
-645 518 5
-6 521 4
-615 269 4
-92 794 3
-713 311 3
-776 168 5
-680 150 5
-299 813 4
-407 248 4
-830 22 5
-450 387 5
-862 748 4
-271 481 3
-486 148 2
-551 732 4
-201 196 4
-14 813 2
-215 229 3
-234 781 2
-48 259 4
-804 380 4
-829 640 3
-865 118 1
-757 426 3
-774 56 2
-286 1411 2
-151 736 4
-425 347 4
-834 148 4
-200 8 4
-588 731 2
-887 578 4
-912 97 4
-936 717 2
-653 1620 2
-521 151 3
-592 192 5
-794 286 3
-347 546 4
-647 831 3
-716 4 2
-139 100 5
-276 854 4
-267 53 4
-682 135 4
-673 898 3
-734 173 3
-416 307 1
-25 174 5
-546 816 3
-648 931 2
-168 125 4
-938 257 5
-863 906 4
-792 25 2
-200 229 5
-639 88 3
-929 174 3
-347 180 5
-707 200 2
-881 216 4
-880 56 5
-526 147 4
-552 815 3
-506 693 4
-756 367 4
-318 796 3
-472 366 4
-82 276 4
-130 890 4
-574 305 3
-500 1226 4
-697 124 5
-10 161 4
-387 194 3
-472 432 5
-555 89 4
-450 583 4
-385 1456 4
-10 488 5
-659 578 3
-64 603 3
-642 420 4
-130 939 4
-130 316 4
-524 265 4
-291 670 5
-28 322 2
-472 82 5
-296 198 5
-774 222 3
-782 1534 2
-833 405 3
-21 977 2
-43 151 4
-158 580 4
-882 559 3
-312 137 3
-916 1220 3
-694 177 5
-847 456 1
-805 50 4
-265 258 4
-534 243 3
-425 679 3
-697 742 3
-493 100 5
-419 223 4
-711 254 2
-533 222 5
-751 658 3
-28 153 3
-160 285 4
-388 258 5
-660 826 3
-214 238 4
-305 317 4
-292 298 4
-267 367 4
-807 1409 4
-499 295 2
-783 343 5
-794 242 5
-280 54 2
-864 22 5
-158 154 4
-181 828 1
-313 225 4
-671 779 3
-846 197 4
-763 955 2
-407 82 3
-327 1 4
-774 423 1
-286 955 5
-585 638 4
-269 1101 4
-357 744 5
-186 269 1
-715 195 4
-846 673 4
-297 498 3
-103 1089 1
-13 636 2
-141 1283 3
-452 82 3
-60 79 4
-936 16 4
-661 272 4
-887 1051 4
-881 756 4
-682 625 3
-883 64 4
-592 221 5
-773 145 3
-796 12 5
-714 252 3
-757 549 5
-303 780 5
-188 732 3
-841 678 4
-218 204 3
-624 111 3
-792 1011 3
-835 928 3
-524 208 5
-747 231 3
-879 118 3
-518 151 3
-733 116 4
-724 329 4
-682 808 4
-712 1503 4
-932 968 4
-261 301 4
-177 173 4
-839 1664 1
-634 405 4
-144 709 4
-639 166 3
-493 192 3
-452 98 5
-718 926 2
-170 326 5
-457 469 4
-405 1139 1
-463 473 4
-18 971 4
-805 196 2
-901 181 4
-405 664 1
-201 50 4
-332 1013 3
-56 993 3
-431 748 4
-802 98 4
-398 230 3
-291 128 4
-217 568 4
-320 79 4
-378 1009 3
-248 114 5
-727 444 2
-592 169 5
-460 124 4
-472 1210 3
-833 128 3
-189 603 5
-474 385 4
-878 168 4
-537 216 3
-417 72 4
-201 46 4
-901 1389 5
-70 746 3
-200 325 5
-674 597 3
-660 83 3
-500 557 3
-487 627 4
-328 1136 4
-317 299 4
-904 274 5
-254 384 1
-562 190 4
-155 332 2
-342 191 5
-840 216 4
-424 681 3
-561 234 3
-709 210 4
-746 144 5
-130 1089 2
-406 11 4
-7 127 5
-840 616 5
-497 405 3
-62 24 4
-833 79 3
-235 85 4
-919 20 1
-896 215 5
-59 735 5
-537 707 4
-480 56 4
-255 294 2
-807 528 4
-186 566 5
-917 405 3
-395 154 5
-478 357 5
-658 100 4
-276 161 3
-616 879 4
-347 819 1
-487 289 2
-886 49 4
-31 299 4
-141 255 4
-600 550 4
-860 301 2
-942 259 4
-458 48 4
-279 184 5
-274 1 4
-694 176 5
-270 1007 5
-201 658 3
-863 330 2
-323 150 4
-807 417 3
-271 815 3
-887 365 5
-916 385 3
-363 506 2
-499 143 3
-115 192 5
-629 172 5
-90 1136 3
-916 217 4
-561 1478 3
-308 54 2
-934 461 4
-223 249 2
-467 100 5
-561 483 4
-676 480 5
-59 99 4
-745 177 3
-474 25 5
-886 50 5
-85 1070 4
-304 237 5
-790 1063 5
-737 428 4
-303 398 1
-138 238 5
-660 1181 1
-117 1057 2
-60 1121 3
-178 1197 4
-95 90 2
-110 29 3
-10 9 4
-592 685 2
-332 841 4
-887 225 4
-452 648 4
-108 748 3
-745 923 3
-49 717 2
-940 98 4
-552 926 2
-707 12 3
-682 1410 3
-784 326 5
-308 826 3
-506 81 1
-449 288 3
-704 480 5
-287 952 4
-232 515 2
-417 388 3
-21 408 5
-451 292 3
-387 463 4
-840 71 3
-694 191 5
-117 1165 3
-841 300 4
-870 272 4
-425 174 3
-880 1215 1
-62 286 3
-407 474 3
-774 96 2
-524 955 1
-269 435 3
-479 190 4
-344 39 3
-903 763 5
-709 17 4
-399 382 3
-417 827 2
-537 262 5
-210 65 4
-279 180 2
-164 689 5
-416 476 5
-865 189 4
-561 505 4
-374 597 4
-755 872 1
-514 176 4
-934 949 3
-729 690 2
-290 43 3
-95 1229 2
-637 289 2
-569 222 3
-585 1266 3
-450 530 3
-477 794 4
-77 252 1
-2 50 5
-234 185 3
-435 67 4
-308 284 4
-344 64 5
-297 724 3
-343 217 3
-288 98 5
-566 12 4
-270 869 1
-21 994 2
-10 234 4
-385 131 4
-344 285 5
-474 431 4
-592 472 1
-291 64 5
-496 217 5
-283 168 5
-407 158 2
-833 928 2
-712 1040 4
-303 1073 4
-37 55 3
-49 346 4
-875 964 4
-305 275 2
-160 484 5
-230 284 1
-274 258 5
-807 227 4
-936 129 4
-234 966 4
-621 25 4
-565 213 4
-346 569 3
-374 2 4
-94 41 3
-787 302 3
-642 292 2
-338 650 5
-616 245 3
-880 623 4
-437 747 4
-354 135 3
-417 715 2
-227 273 3
-181 1317 1
-389 467 3
-38 318 3
-506 586 2
-561 80 2
-419 705 5
-554 864 4
-40 269 1
-62 216 4
-141 346 1
-301 215 5
-885 386 2
-655 1628 2
-301 402 2
-532 96 5
-417 118 4
-291 93 4
-167 698 4
-886 659 4
-511 948 3
-699 121 3
-671 89 5
-864 12 5
-843 153 3
-524 612 3
-108 740 3
-339 1248 3
-708 762 5
-96 83 3
-864 174 5
-279 515 3
-705 862 1
-493 876 1
-498 222 3
-936 255 5
-389 58 4
-429 11 4
-234 76 2
-313 117 4
-23 234 2
-181 1245 1
-721 242 3
-886 237 4
-645 506 5
-373 281 3
-850 568 5
-716 318 5
-905 302 5
-608 1009 4
-234 195 2
-222 423 4
-130 752 5
-320 566 3
-224 147 3
-311 118 3
-315 709 4
-586 203 3
-214 276 3
-561 1018 3
-116 260 2
-42 72 3
-727 7 2
-82 112 1
-26 286 3
-518 237 4
-456 56 5
-758 211 4
-416 658 5
-472 416 3
-250 331 3
-141 250 4
-612 127 2
-556 192 5
-844 625 3
-840 503 4
-83 174 5
-195 921 3
-525 472 2
-130 347 4
-843 465 2
-479 203 3
-648 429 4
-144 87 5
-278 98 4
-92 595 3
-416 196 5
-933 230 3
-354 747 2
-758 484 5
-826 1231 3
-694 203 4
-456 161 3
-348 1 4
-867 956 4
-699 276 3
-861 286 4
-381 283 5
-389 393 2
-854 431 3
-608 735 4
-562 485 5
-453 1273 2
-889 33 5
-894 471 4
-339 298 2
-782 1038 4
-526 886 3
-77 357 3
-576 181 4
-868 159 2
-573 528 4
-652 125 2
-407 162 4
-75 301 4
-38 403 1
-271 657 4
-936 137 4
-69 79 4
-537 445 3
-318 208 4
-509 326 4
-938 676 3
-286 41 2
-227 117 2
-200 96 5
-472 501 3
-567 293 5
-52 405 4
-342 1057 2
-342 1007 4
-506 47 4
-852 1615 2
-436 785 2
-747 94 4
-514 190 5
-831 64 5
-152 845 3
-889 26 4
-569 283 4
-452 153 4
-483 222 3
-123 435 5
-503 181 5
-416 231 3
-308 410 4
-922 15 4
-318 540 4
-174 286 5
-758 455 4
-583 239 2
-466 908 4
-447 815 3
-790 722 3
-387 514 3
-659 526 5
-405 1307 1
-551 121 5
-224 178 4
-405 1271 2
-900 1028 2
-672 756 2
-796 164 3
-389 95 3
-456 743 2
-680 294 4
-726 898 2
-505 11 4
-458 182 4
-805 436 3
-113 319 2
-144 14 4
-409 714 3
-489 260 3
-684 1301 3
-829 13 4
-394 156 4
-422 333 4
-63 741 3
-222 746 5
-846 388 3
-593 735 4
-286 91 4
-886 7 5
-536 1 5
-286 204 3
-119 100 5
-569 1 4
-747 588 5
-303 1047 2
-567 640 4
-7 571 3
-861 740 4
-789 762 3
-13 784 1
-389 82 4
-899 168 4
-291 974 1
-776 524 5
-778 209 4
-244 92 4
-408 294 5
-194 433 3
-514 392 4
-372 164 4
-194 160 2
-148 116 5
-389 28 4
-401 88 4
-591 956 4
-660 1183 1
-92 1033 2
-342 188 3
-655 1641 3
-711 180 4
-560 1014 4
-655 443 4
-325 179 5
-940 549 2
-116 322 2
-328 1112 4
-144 135 5
-87 796 4
-130 222 4
-757 11 4
-200 1219 3
-429 697 3
-450 225 4
-796 491 4
-7 192 4
-409 210 4
-409 1512 5
-735 288 4
-490 596 1
-145 879 5
-296 179 4
-393 792 1
-881 28 5
-37 7 4
-414 325 3
-308 675 4
-345 117 4
-256 472 4
-829 475 4
-919 315 3
-42 274 5
-123 483 4
-896 172 5
-650 659 3
-535 269 4
-472 760 5
-472 11 5
-894 1501 4
-454 144 4
-526 343 3
-526 293 5
-605 64 5
-871 575 5
-472 378 4
-13 28 5
-43 25 5
-296 257 5
-366 218 3
-506 665 2
-943 406 3
-859 111 4
-474 513 5
-402 235 3
-472 96 5
-592 931 1
-815 87 5
-889 56 5
-254 22 4
-393 794 4
-568 1125 4
-243 194 4
-727 1016 3
-807 1084 4
-875 42 4
-517 269 3
-919 70 4
-130 44 4
-440 1105 5
-6 301 2
-457 747 4
-264 70 4
-64 405 3
-405 1029 1
-308 1411 4
-268 100 3
-653 214 3
-476 1037 1
-110 734 2
-303 1160 2
-524 58 4
-276 142 3
-222 588 4
-399 561 2
-473 321 2
-829 458 3
-60 608 5
-622 213 5
-286 248 5
-640 85 5
-470 9 5
-59 42 5
-284 346 4
-363 393 4
-72 198 5
-934 675 4
-532 1470 5
-94 1210 3
-795 70 3
-623 222 4
-927 393 5
-808 748 4
-778 216 3
-487 27 5
-399 179 3
-96 1154 5
-231 471 5
-176 257 1
-405 504 2
-87 182 4
-646 272 4
-699 880 3
-83 783 4
-11 268 5
-385 508 2
-528 541 3
-252 276 5
-266 508 4
-92 651 4
-577 405 3
-897 235 3
-797 257 5
-798 395 3
-697 343 4
-916 143 3
-269 509 4
-92 129 4
-426 603 5
-633 147 4
-442 12 4
-897 405 5
-489 890 5
-215 215 3
-707 194 4
-334 1137 4
-838 72 4
-456 588 3
-157 273 5
-315 23 5
-618 143 4
-924 216 4
-263 31 4
-10 482 4
-711 219 2
-344 276 4
-920 292 3
-540 257 4
-507 597 5
-712 400 3
-71 174 2
-58 127 4
-776 1219 3
-294 926 3
-246 184 4
-807 624 3
-87 194 5
-343 187 4
-445 823 1
-110 692 4
-90 753 4
-157 290 4
-246 1073 4
-478 81 4
-887 118 5
-554 125 3
-559 652 4
-454 612 3
-468 856 4
-268 286 5
-386 127 5
-385 1069 4
-592 886 3
-130 934 4
-348 276 3
-156 77 2
-916 250 4
-532 721 4
-629 4 3
-436 83 5
-820 288 5
-94 42 4
-16 92 4
-881 826 1
-874 20 3
-533 988 2
-276 696 2
-821 15 5
-863 258 5
-562 191 5
-495 200 5
-645 558 4
-62 1016 4
-457 161 4
-299 259 3
-145 993 3
-301 1028 5
-629 690 2
-328 55 4
-298 526 5
-269 436 3
-269 58 2
-879 685 4
-417 797 3
-1 260 1
-42 48 5
-343 237 4
-606 939 4
-64 778 5
-72 69 4
-279 1142 1
-344 183 5
-94 1147 4
-237 183 5
-313 673 4
-932 841 2
-374 932 1
-296 250 2
-881 139 3
-552 756 2
-227 221 4
-346 546 4
-943 168 2
-883 399 5
-416 184 4
-561 191 3
-158 174 5
-417 1210 2
-85 414 4
-344 173 5
-269 616 4
-621 233 3
-294 678 2
-479 471 4
-450 526 4
-543 210 3
-912 443 4
-468 153 5
-280 845 3
-60 480 4
-870 181 4
-97 183 5
-102 271 2
-308 709 3
-20 98 3
-747 333 4
-425 326 1
-328 331 4
-554 222 4
-666 25 3
-94 328 3
-7 528 5
-610 568 4
-28 56 5
-408 272 4
-756 235 3
-577 173 5
-799 331 4
-466 403 3
-537 176 2
-493 196 4
-901 546 4
-429 409 2
-486 282 2
-321 193 3
-246 145 1
-654 739 4
-716 660 4
-869 118 1
-295 115 5
-178 484 4
-346 431 5
-539 367 3
-45 472 3
-397 591 4
-497 625 3
-268 474 5
-447 770 3
-162 151 3
-580 1014 3
-705 28 4
-339 806 4
-417 513 5
-87 63 4
-63 678 2
-880 931 3
-880 23 5
-169 243 3
-389 558 4
-486 9 5
-450 618 4
-586 173 3
-185 15 3
-13 869 3
-884 86 3
-94 64 5
-559 322 4
-100 286 3
-70 483 5
-505 125 3
-338 1124 4
-867 89 5
-655 208 3
-763 625 4
-643 550 3
-117 1 4
-530 191 5
-49 1069 3
-597 293 5
-404 1238 3
-385 1037 1
-661 161 4
-493 647 4
-842 344 1
-83 623 4
-179 340 4
-153 187 2
-5 435 4
-644 307 4
-870 435 3
-95 1219 1
-89 237 4
-54 250 4
-251 257 3
-389 94 2
-10 170 4
-293 553 3
-913 273 3
-715 1215 1
-560 1008 3
-472 455 4
-411 210 5
-523 9 4
-343 930 1
-870 100 4
-591 709 4
-727 12 5
-632 11 4
-567 527 3
-854 945 3
-297 294 3
-93 14 4
-509 301 2
-532 315 3
-151 611 4
-669 480 5
-581 813 5
-10 489 4
-72 230 1
-532 538 4
-378 225 3
-116 1216 3
-305 56 1
-466 231 1
-221 100 5
-922 1110 4
-184 378 4
-666 236 4
-618 501 4
-442 230 3
-748 137 3
-436 1048 2
-530 100 4
-786 111 5
-541 196 4
-174 950 3
-932 443 4
-268 456 2
-870 1267 2
-907 278 5
-740 328 3
-539 483 5
-299 478 4
-790 203 4
-659 199 4
-548 298 4
-442 449 2
-95 660 5
-592 433 5
-413 258 4
-374 472 2
-553 98 5
-749 174 5
-13 761 4
-488 211 4
-379 12 5
-916 1098 4
-215 151 5
-599 282 5
-682 1232 2
-648 926 3
-213 735 5
-566 1028 2
-23 56 4
-204 301 4
-889 386 3
-70 91 3
-833 443 3
-244 1168 4
-349 291 3
-320 685 4
-880 1165 2
-660 47 2
-739 603 4
-660 362 2
-425 743 4
-870 169 4
-498 50 4
-293 642 3
-698 419 3
-49 53 4
-526 919 3
-216 3 4
-586 559 5
-763 39 4
-650 202 3
-574 333 3
-479 71 1
-295 213 5
-379 216 4
-640 941 5
-916 135 4
-363 760 1
-868 216 2
-210 174 5
-116 253 3
-594 520 4
-308 430 4
-557 271 4
-49 185 5
-311 768 2
-884 268 4
-715 118 2
-434 148 3
-299 1379 3
-203 993 3
-23 155 3
-516 181 4
-648 781 4
-379 127 5
-698 210 5
-655 318 4
-457 417 4
-518 628 5
-274 237 4
-617 145 1
-916 636 3
-688 339 5
-253 98 5
-229 272 3
-630 988 2
-666 505 4
-104 748 2
-903 129 3
-13 589 3
-425 178 3
-846 63 3
-314 761 4
-201 588 4
-798 827 4
-62 21 3
-60 50 5
-216 153 4
-406 478 4
-627 713 2
-201 708 4
-715 288 4
-618 1063 3
-916 204 3
-354 558 4
-416 356 5
-655 306 3
-506 538 3
-378 94 3
-452 168 4
-779 509 2
-13 179 2
-18 845 3
-201 483 3
-632 404 5
-892 230 4
-747 301 1
-592 682 4
-189 863 4
-772 312 4
-15 938 3
-724 307 3
-193 1168 4
-331 454 3
-627 554 2
-788 133 5
-714 477 2
-94 219 4
-852 826 3
-417 109 2
-347 176 3
-497 1210 4
-491 900 5
-407 650 2
-795 588 5
-403 288 4
-267 69 4
-455 173 4
-890 632 5
-642 568 4
-602 880 4
-880 191 5
-776 91 4
-222 35 1
-262 402 4
-436 92 3
-406 121 5
-729 272 4
-298 91 2
-221 12 5
-880 1016 4
-484 472 4
-715 564 2
-154 642 3
-551 302 3
-21 242 3
-617 100 4
-650 162 3
-292 602 4
-880 1258 3
-773 233 1
-42 219 1
-44 144 4
-655 1418 4
-686 588 4
-313 181 4
-541 560 3
-295 137 4
-532 373 3
-315 56 5
-796 516 4
-181 1047 2
-782 905 4
-496 743 2
-197 322 3
-119 995 4
-429 178 4
-416 98 5
-566 1005 5
-90 18 3
-92 22 3
-269 131 5
-324 879 4
-49 433 5
-942 272 5
-548 471 5
-303 25 4
-234 482 4
-303 1337 1
-320 292 3
-304 343 3
-625 428 5
-316 197 4
-642 1182 2
-275 229 3
-328 879 3
-303 155 3
-334 173 4
-94 89 3
-537 1103 4
-87 409 3
-727 771 3
-679 327 4
-112 306 5
-926 272 5
-87 535 4
-741 1029 1
-501 369 4
-85 521 3
-515 900 4
-405 674 1
-82 409 1
-385 209 4
-385 156 4
-326 194 4
-840 300 3
-251 1 4
-224 570 4
-168 845 4
-226 180 4
-886 288 4
-758 66 3
-408 539 1
-353 245 4
-562 458 2
-300 328 3
-205 313 3
-796 409 3
-566 523 4
-450 3 4
-648 105 3
-450 1172 5
-425 325 3
-428 312 4
-13 606 4
-694 28 4
-222 739 4
-773 61 5
-276 631 3
-291 356 4
-196 108 4
-703 591 4
-381 432 5
-83 781 4
-711 114 5
-416 213 5
-1 16 5
-406 197 4
-184 606 5
-763 127 4
-238 181 3
-773 196 4
-102 1239 2
-60 181 4
-916 727 4
-529 690 3
-636 596 5
-601 121 2
-843 298 2
-489 321 3
-193 627 4
-299 179 4
-386 118 3
-903 1008 3
-485 748 2
-620 420 3
-474 42 4
-642 1219 4
-262 71 4
-406 70 3
-838 487 4
-907 8 3
-227 116 4
-775 315 5
-660 358 2
-503 88 4
-313 391 3
-527 514 5
-393 121 4
-643 172 5
-6 9 4
-771 202 4
-85 606 4
-582 121 3
-327 357 4
-564 292 4
-639 647 3
-903 509 4
-868 136 5
-107 323 1
-916 960 4
-116 582 3
-788 44 4
-248 235 3
-98 435 5
-666 118 3
-200 188 4
-446 301 3
-857 300 3
-454 317 4
-795 425 3
-749 194 5
-747 409 1
-896 836 3
-864 805 4
-892 423 5
-478 28 3
-577 110 4
-740 286 5
-524 414 4
-6 523 5
-201 1128 4
-308 962 4
-241 750 5
-847 220 4
-621 62 4
-401 83 4
-436 790 3
-21 817 3
-426 191 4
-308 295 3
-342 382 3
-815 993 2
-397 483 5
-158 96 4
-248 64 5
-7 357 5
-59 789 4
-102 226 2
-397 182 5
-354 269 4
-174 699 5
-450 520 5
-878 20 2
-829 105 3
-239 96 5
-363 1009 2
-301 227 3
-666 255 4
-638 504 2
-735 741 2
-870 239 3
-201 443 3
-297 716 3
-49 1075 2
-472 140 3
-711 699 5
-532 451 4
-20 50 3
-333 168 4
-686 430 4
-216 658 3
-263 294 2
-393 540 3
-436 747 5
-815 57 5
-638 222 4
-450 647 4
-70 228 5
-653 1023 3
-294 872 4
-233 89 3
-160 248 5
-406 501 5
-683 588 4
-617 854 1
-387 277 4
-880 3 1
-234 605 3
-552 873 3
-880 24 3
-435 443 3
-919 313 5
-226 480 4
-491 294 2
-284 300 3
-406 414 2
-183 144 3
-606 111 4
-5 50 4
-712 722 3
-43 732 4
-374 928 1
-671 570 3
-465 705 4
-878 153 5
-561 603 4
-864 781 3
-146 331 5
-497 826 3
-117 150 4
-883 275 4
-26 235 2
-258 751 5
-846 428 3
-664 705 4
-293 528 4
-846 131 3
-600 92 3
-896 141 3
-398 125 3
-256 1150 5
-665 92 4
-44 260 4
-117 33 4
-622 135 4
-650 63 2
-665 699 4
-407 239 4
-530 163 3
-896 239 4
-479 168 5
-908 195 4
-541 225 4
-601 928 1
-178 232 5
-532 252 4
-500 919 3
-834 307 4
-374 96 4
-663 696 3
-876 511 5
-721 229 5
-804 552 4
-130 47 3
-825 50 4
-401 1 2
-199 242 5
-450 174 5
-659 528 4
-181 299 1
-901 679 4
-682 693 3
-69 1017 5
-181 931 1
-860 262 4
-312 474 5
-5 411 1
-617 7 3
-663 147 3
-271 756 2
-518 14 3
-297 24 4
-83 210 5
-210 135 5
-256 106 4
-210 40 3
-102 823 3
-455 292 3
-501 845 3
-538 240 2
-56 473 2
-13 798 2
-655 28 3
-637 333 3
-702 449 3
-887 8 4
-534 237 4
-176 93 5
-535 609 4
-186 322 5
-559 427 4
-346 578 2
-158 399 3
-625 516 3
-588 417 5
-938 245 3
-912 496 4
-405 739 2
-592 291 3
-391 98 4
-783 258 4
-845 242 4
-807 554 4
-387 549 5
-561 514 4
-248 290 3
-422 1 3
-497 441 2
-378 64 4
-95 180 3
-276 1228 1
-504 719 3
-844 588 4
-812 333 5
-556 604 5
-235 346 4
-453 410 4
-614 121 4
-472 603 5
-825 122 1
-769 597 2
-201 1131 5
-233 194 4
-579 210 3
-422 50 4
-694 157 4
-747 124 5
-480 190 5
-11 727 3
-176 293 5
-234 1050 3
-916 60 4
-903 1 3
-416 693 3
-174 395 1
-532 833 4
-428 269 5
-883 202 4
-580 250 5
-920 350 4
-551 95 5
-578 288 3
-5 1 4
-180 40 4
-896 483 3
-562 82 5
-899 95 5
-276 157 5
-287 347 4
-430 100 5
-378 254 1
-415 322 4
-506 836 4
-677 455 5
-474 83 3
-59 566 4
-543 239 2
-656 689 2
-233 50 3
-498 381 3
-181 1355 1
-374 227 4
-892 117 4
-620 674 3
-72 64 5
-561 203 4
-919 222 3
-669 657 5
-130 77 5
-813 898 1
-279 465 5
-487 651 5
-511 300 4
-500 559 4
-1 14 5
-347 151 3
-274 471 4
-750 338 3
-639 727 2
-347 97 4
-435 11 5
-554 405 4
-625 210 3
-851 1051 2
-1 202 5
-399 542 3
-663 876 3
-140 304 4
-416 274 4
-892 204 4
-71 289 2
-18 172 3
-887 90 5
-843 429 4
-630 125 3
-506 135 5
-468 172 4
-293 468 2
-405 644 3
-193 301 4
-639 655 3
-875 478 4
-717 831 3
-83 97 4
-279 117 5
-389 118 2
-118 427 5
-437 418 3
-280 750 5
-35 328 3
-135 581 4
-312 414 3
-458 126 4
-913 12 4
-7 634 5
-484 720 4
-253 485 5
-234 140 2
-774 393 1
-401 481 3
-191 328 3
-250 69 5
-1 54 3
-23 516 4
-402 111 4
-883 550 3
-771 128 2
-419 100 5
-924 50 5
-401 356 4
-455 65 3
-291 1253 3
-542 109 4
-497 227 2
-869 276 4
-177 508 4
-504 546 4
-481 144 4
-115 117 4
-633 665 3
-406 605 5
-200 1139 3
-715 761 3
-587 688 3
-634 122 3
-840 566 5
-314 147 4
-847 735 4
-387 380 2
-449 310 3
-843 526 3
-884 269 5
-592 984 1
-705 298 5
-457 203 4
-479 739 1
-745 515 4
-85 229 3
-291 290 4
-83 71 3
-347 411 5
-894 336 3
-924 923 5
-604 185 2
-308 664 5
-143 1038 3
-313 403 3
-486 408 3
-427 359 5
-699 100 4
-913 60 3
-409 175 4
-94 12 4
-825 125 5
-707 1142 1
-142 186 4
-279 294 2
-886 1228 2
-682 568 3
-843 172 3
-394 802 1
-535 137 4
-178 83 4
-116 127 5
-23 131 4
-311 192 3
-618 195 3
-429 805 3
-587 292 3
-871 202 4
-918 430 1
-612 202 2
-295 144 4
-650 480 5
-329 879 2
-800 457 2
-872 748 3
-798 110 4
-194 1220 3
-325 174 2
-617 320 5
-325 835 5
-312 1192 3
-294 248 5
-606 156 4
-409 479 5
-72 187 4
-343 385 3
-417 182 4
-642 769 5
-886 184 4
-848 603 5
-847 125 3
-457 192 5
-316 614 2
-327 558 4
-680 20 4
-655 912 3
-833 122 2
-933 22 5
-716 340 3
-507 50 5
-10 486 4
-424 294 5
-59 403 5
-591 285 5
-895 117 3
-524 647 3
-833 802 1
-487 218 2
-328 582 5
-892 11 3
-260 362 5
-10 484 5
-537 1245 3
-369 316 5
-880 195 4
-749 448 2
-360 210 4
-295 208 5
-270 230 3
-323 144 4
-11 290 3
-903 238 5
-429 941 3
-883 313 3
-537 3 2
-592 71 4
-343 631 4
-451 325 3
-346 196 3
-540 628 3
-379 294 3
-213 70 3
-903 234 4
-363 50 5
-894 190 5
-194 1058 2
-399 93 3
-74 333 4
-334 1006 3
-204 191 4
-13 911 2
-773 568 1
-545 132 4
-548 346 4
-269 462 3
-932 399 4
-119 410 1
-213 121 5
-493 264 3
-881 504 3
-419 191 4
-65 202 4
-896 227 4
-632 684 5
-318 179 4
-293 148 1
-454 509 2
-463 1017 2
-324 875 3
-671 161 5
-593 568 4
-833 1029 1
-388 680 5
-158 194 5
-664 229 3
-647 71 4
-843 474 3
-459 696 4
-642 168 5
-743 340 3
-95 498 3
-532 210 5
-279 163 5
-207 509 4
-671 802 3
-617 1316 1
-352 55 1
-691 1172 5
-516 523 3
-115 9 5
-216 367 3
-707 1022 3
-429 755 3
-374 202 3
-280 99 2
-417 47 3
-108 1 4
-535 1136 4
-880 584 3
-492 124 4
-271 1046 4
-94 390 5
-749 501 4
-518 1017 3
-293 386 2
-838 705 5
-500 93 4
-225 606 5
-244 537 5
-76 223 2
-666 121 3
-625 173 3
-130 1047 5
-838 169 4
-474 1124 4
-95 514 2
-789 276 5
-838 258 5
-782 1611 3
-423 591 5
-655 300 3
-765 1009 5
-774 947 2
-263 521 3
-311 1232 4
-319 880 4
-336 1017 5
-899 499 3
-255 564 1
-95 143 4
-813 538 3
-537 135 5
-472 1079 4
-13 601 4
-545 226 3
-468 461 4
-880 1000 3
-178 97 5
-435 797 3
-102 947 3
-80 213 3
-437 174 5
-661 443 4
-498 317 3
-201 190 4
-411 73 4
-766 662 3
-751 483 5
-244 739 3
-457 134 5
-405 727 1
-429 431 5
-655 30 5
-488 8 3
-82 455 4
-784 321 3
-933 568 2
-802 447 2
-683 887 4
-230 265 5
-162 237 4
-916 182 3
-828 57 3
-389 686 3
-244 949 4
-21 118 1
-427 874 5
-655 695 3
-551 693 5
-466 546 4
-545 944 4
-495 521 5
-342 42 3
-201 197 4
-642 1179 3
-391 213 4
-566 89 4
-741 288 4
-537 180 4
-13 224 4
-387 295 3
-62 15 2
-709 11 5
-308 736 3
-251 295 4
-405 1425 1
-183 54 2
-864 176 5
-286 683 5
-686 48 5
-733 16 3
-85 842 3
-35 261 3
-271 392 3
-599 476 4
-416 549 4
-568 79 4
-312 684 5
-676 1654 1
-479 135 4
-296 23 5
-312 430 5
-286 1039 5
-913 216 4
-269 478 4
-892 729 4
-699 749 3
-882 11 4
-322 654 5
-291 100 5
-313 745 3
-246 433 5
-478 410 3
-804 631 3
-363 102 4
-222 1438 4
-910 134 3
-472 746 5
-638 186 5
-551 58 5
-407 746 4
-442 834 2
-60 736 5
-650 234 4
-589 294 5
-702 380 4
-71 302 3
-291 715 5
-72 45 5
-189 120 1
-144 93 1
-682 184 4
-910 56 4
-95 89 3
-608 92 3
-539 269 5
-87 179 4
-495 226 4
-26 292 3
-792 111 3
-387 1199 5
-393 196 4
-399 323 1
-658 169 5
-877 170 5
-847 826 3
-455 200 5
-618 148 3
-567 246 4
-452 285 3
-663 975 4
-661 318 5
-423 316 4
-328 723 3
-551 79 5
-263 528 4
-648 83 4
-653 1267 1
-537 221 3
-621 176 3
-393 751 2
-75 1059 1
-407 98 5
-606 95 4
-374 544 1
-533 282 4
-437 13 4
-7 427 5
-37 121 2
-532 633 5
-389 105 3
-95 649 4
-728 15 4
-872 925 4
-940 168 3
-279 946 3
-553 434 3
-896 658 4
-907 633 5
-10 48 4
-492 527 5
-939 1190 5
-474 151 3
-416 1135 2
-615 699 3
-798 480 3
-263 127 4
-389 475 5
-325 357 4
-907 1119 5
-381 132 5
-487 471 3
-407 655 4
-102 272 3
-23 98 5
-436 537 4
-435 943 3
-438 845 4
-232 91 5
-158 8 5
-94 100 5
-738 135 5
-659 202 4
-393 727 3
-618 185 5
-932 213 3
-450 301 4
-656 346 3
-22 110 1
-640 578 3
-862 866 4
-450 699 4
-881 477 4
-693 143 4
-388 682 4
-367 800 4
-452 475 2
-13 424 1
-833 68 4
-148 521 1
-626 264 1
-411 58 3
-21 981 2
-648 929 4
-18 775 3
-500 289 4
-877 173 4
-413 321 3
-749 380 3
-500 699 3
-324 294 5
-366 98 5
-298 153 3
-538 963 4
-406 1021 5
-655 135 4
-279 982 3
-557 270 3
-172 514 3
-758 447 4
-99 619 4
-607 435 3
-312 91 3
-840 163 4
-271 248 4
-897 928 5
-415 180 5
-919 319 3
-256 1119 3
-374 318 2
-668 69 1
-940 427 5
-234 1221 4
-195 67 2
-332 554 3
-913 132 3
-348 476 4
-1 101 2
-318 186 5
-738 175 4
-605 22 4
-387 265 4
-542 648 4
-642 1047 3
-533 748 3
-450 157 3
-201 334 4
-798 270 4
-903 282 4
-894 1462 3
-387 117 3
-379 654 5
-290 435 3
-585 275 4
-703 742 3
-624 924 4
-568 30 4
-59 705 4
-429 156 4
-537 972 3
-44 756 3
-497 174 4
-830 89 5
-385 1070 5
-224 313 5
-579 333 4
-655 271 3
-502 681 1
-864 569 3
-18 216 4
-701 750 5
-234 389 3
-483 462 3
-542 772 4
-911 1039 4
-399 210 3
-308 171 4
-56 42 4
-566 177 4
-881 129 4
-56 423 5
-341 881 5
-151 514 4
-109 380 5
-694 31 4
-554 209 4
-671 356 3
-582 831 2
-380 514 2
-927 158 2
-902 479 4
-449 127 5
-754 922 3
-455 511 5
-455 47 2
-197 576 4
-482 298 4
-659 210 5
-543 391 3
-232 589 3
-400 286 4
-1 226 3
-417 89 5
-579 7 3
-683 914 2
-181 1255 1
-524 1553 3
-622 276 4
-637 301 1
-7 78 3
-92 282 4
-806 133 5
-622 674 2
-618 99 3
-659 496 5
-483 271 3
-132 12 4
-903 521 5
-422 567 3
-863 342 1
-699 717 1
-527 324 3
-650 658 3
-102 288 2
-650 203 3
-94 559 4
-763 1098 3
-20 763 1
-716 234 5
-16 812 2
-747 494 5
-537 736 3
-795 367 3
-279 978 1
-184 845 3
-472 196 4
-209 688 1
-848 423 4
-321 131 4
-297 357 4
-258 326 5
-379 496 5
-78 871 3
-369 948 2
-425 1110 1
-389 384 2
-758 121 2
-447 153 4
-234 483 5
-890 516 2
-804 928 4
-277 282 4
-186 121 2
-449 179 4
-308 9 4
-276 452 3
-137 172 5
-13 815 4
-693 218 4
-269 197 5
-270 25 5
-62 318 5
-446 322 3
-774 431 4
-506 175 5
-394 42 4
-9 507 4
-641 338 3
-551 1621 1
-115 13 5
-276 458 4
-197 233 4
-758 196 4
-267 214 4
-419 269 4
-315 31 3
-672 321 4
-184 82 3
-10 12 5
-113 321 3
-178 628 4
-821 151 4
-18 4 3
-449 86 4
-10 498 5
-781 288 2
-751 193 5
-669 517 3
-347 156 5
-457 15 4
-763 418 4
-7 307 5
-235 319 4
-450 749 4
-181 975 2
-479 584 3
-806 176 5
-244 80 3
-887 204 5
-816 243 4
-807 520 5
-181 104 1
-604 184 3
-87 1028 4
-751 270 4
-399 926 2
-212 631 5
-870 461 4
-402 151 5
-13 227 5
-246 38 2
-237 199 4
-618 70 3
-453 117 4
-194 625 3
-297 7 4
-318 1032 3
-880 49 3
-448 316 1
-43 580 3
-286 381 5
-588 463 4
-406 701 5
-215 354 4
-884 258 5
-186 554 1
-119 235 5
-307 427 3
-288 593 2
-59 228 4
-719 66 3
-130 1207 1
-585 207 5
-682 735 4
-472 931 2
-732 243 5
-13 242 2
-178 762 3
-405 1109 1
-782 271 2
-189 1021 5
-870 582 5
-569 508 3
-543 192 4
-515 300 5
-653 132 3
-932 520 4
-719 532 3
-718 597 5
-458 289 2
-829 339 2
-186 820 2
-7 612 5
-686 181 4
-450 568 4
-350 228 4
-453 717 2
-550 237 3
-454 661 4
-193 781 3
-303 801 1
-790 427 4
-271 242 4
-760 288 4
-870 1221 3
-655 23 3
-220 300 5
-593 274 3
-561 47 4
-264 856 3
-92 376 3
-211 181 1
-763 157 4
-377 100 3
-791 275 5
-334 290 3
-416 746 5
-653 692 2
-764 273 3
-833 161 1
-447 234 4
-7 25 3
-303 470 4
-715 231 3
-90 709 5
-794 118 2
-498 179 4
-505 161 3
-7 463 4
-592 268 5
-422 1007 4
-72 191 5
-338 275 5
-860 900 3
-880 1058 2
-280 507 3
-630 595 5
-429 86 5
-538 58 4
-838 204 4
-129 748 2
-741 25 3
-95 170 5
-524 607 3
-798 393 3
-751 480 4
-264 367 4
-334 100 5
-109 55 2
-502 343 5
-269 762 1
-573 10 4
-592 261 1
-85 515 5
-164 411 2
-519 266 5
-463 269 5
-701 328 4
-865 100 4
-908 192 2
-749 105 1
-920 682 3
-96 173 3
-529 333 4
-75 322 1
-389 412 3
-521 343 3
-708 1117 4
-122 191 5
-1 44 5
-44 209 5
-438 301 4
-95 153 5
-417 99 4
-582 742 3
-405 1509 1
-790 561 3
-749 686 4
-186 356 5
-476 384 4
-786 70 4
-271 194 5
-312 639 5
-537 539 1
-711 181 4
-454 956 2
-727 879 4
-659 794 3
-222 151 3
-484 393 1
-710 172 4
-14 628 5
-525 332 4
-193 155 4
-468 926 2
-379 566 4
-223 274 4
-787 245 3
-288 346 5
-642 871 3
-130 824 3
-313 505 5
-294 293 4
-846 747 3
-771 768 4
-269 200 4
-575 168 5
-334 591 4
-663 1067 3
-805 1110 5
-815 95 3
-336 785 1
-602 118 3
-144 411 4
-325 483 5
-467 455 3
-760 25 2
-907 121 4
-527 116 4
-411 709 5
-380 770 3
-125 175 2
-867 23 5
-497 731 3
-399 1139 4
-398 66 4
-349 125 4
-387 20 4
-121 315 4
-214 127 4
-23 385 4
-207 483 5
-268 2 2
-683 325 2
-355 324 4
-67 871 3
-922 655 2
-523 242 5
-346 1231 3
-474 678 2
-860 100 4
-595 544 3
-214 42 5
-460 1011 4
-921 405 3
-942 216 4
-921 1287 1
-803 303 4
-151 507 5
-193 1 4
-474 72 3
-727 419 2
-758 441 3
-23 219 1
-345 405 4
-648 477 3
-838 96 4
-608 144 4
-160 123 4
-152 66 5
-487 1217 3
-588 283 4
-270 258 3
-92 281 3
-736 323 1
-417 164 3
-198 15 3
-119 9 4
-514 13 3
-851 50 5
-621 385 5
-239 633 5
-757 260 3
-412 23 4
-545 395 4
-887 284 4
-474 480 5
-297 201 4
-379 88 4
-699 1033 4
-60 160 4
-675 937 1
-753 322 3
-387 324 4
-434 1060 3
-397 65 2
-447 148 4
-181 547 1
-683 248 4
-337 121 5
-605 1 4
-466 117 5
-489 749 4
-343 197 4
-527 87 3
-796 467 3
-141 245 3
-833 154 5
-679 223 5
-230 51 4
-843 145 3
-141 100 4
-59 473 3
-151 660 4
-85 79 3
-518 100 4
-201 204 4
-279 1435 3
-13 443 4
-716 122 2
-815 529 5
-271 191 5
-92 65 4
-627 1478 3
-505 71 4
-533 83 2
-607 487 4
-234 1460 3
-738 79 3
-472 1011 4
-33 329 4
-79 246 5
-311 527 4
-499 484 4
-465 300 3
-815 191 5
-303 742 4
-527 200 3
-11 168 3
-504 1135 4
-21 258 4
-756 161 3
-351 322 5
-644 289 1
-65 98 4
-880 841 3
-94 386 4
-846 610 4
-104 237 3
-682 542 2
-495 448 5
-932 1456 4
-406 168 3
-524 742 3
-533 148 3
-921 288 3
-496 156 3
-399 148 4
-405 1148 1
-805 631 5
-139 302 3
-408 313 4
-486 258 5
-632 161 3
-398 159 3
-424 1084 5
-1 221 5
-514 14 3
-313 628 4
-389 133 5
-320 340 2
-924 701 4
-205 243 2
-24 132 3
-582 240 4
-13 232 3
-561 162 3
-666 197 4
-896 473 2
-597 990 2
-792 291 2
-854 23 4
-618 56 4
-399 274 3
-92 546 2
-290 133 3
-379 528 5
-1 236 4
-606 284 4
-867 204 4
-180 53 5
-323 23 5
-880 124 5
-758 204 4
-886 1231 3
-304 298 5
-229 340 4
-389 420 3
-130 763 5
-645 340 4
-251 265 3
-624 250 4
-472 186 5
-901 275 3
-379 461 4
-329 185 3
-450 180 4
-450 1221 5
-339 47 4
-655 303 4
-391 705 5
-644 276 4
-334 710 3
-570 690 3
-450 1311 4
-363 428 5
-320 1210 4
-267 22 4
-49 737 1
-342 723 3
-376 427 4
-654 169 5
-448 288 1
-622 94 2
-70 229 3
-276 581 4
-424 989 2
-708 1023 3
-746 455 4
-699 321 3
-672 284 4
-60 96 4
-378 663 3
-94 622 3
-235 1193 4
-497 940 2
-856 750 5
-125 69 4
-136 237 4
-268 200 4
-749 406 4
-836 750 3
-296 228 4
-796 228 5
-102 153 2
-771 91 4
-655 975 3
-405 186 5
-942 659 5
-314 1267 3
-189 248 4
-197 1228 4
-8 187 4
-527 475 3
-796 402 5
-610 489 4
-320 291 4
-619 391 3
-577 161 5
-43 269 5
-8 684 4
-832 873 2
-805 47 5
-267 679 4
-933 1188 1
-10 696 4
-421 331 2
-889 1231 3
-99 815 2
-454 511 3
-405 462 2
-772 879 4
-741 699 4
-77 191 3
-690 73 2
-391 507 4
-290 199 3
-498 435 3
-21 569 3
-843 95 2
-171 270 4
-303 4 4
-620 993 5
-429 154 3
-727 431 4
-567 124 4
-13 88 4
-537 950 3
-462 259 3
-684 202 4
-435 944 2
-548 147 5
-296 137 4
-851 892 2
-764 173 3
-913 64 5
-796 540 2
-887 926 5
-10 168 4
-655 1084 3
-62 443 3
-601 455 4
-128 422 4
-297 946 2
-867 181 5
-194 239 3
-361 170 5
-405 716 1
-615 286 4
-862 172 5
-314 202 5
-627 288 3
-625 480 4
-158 120 1
-484 135 4
-763 275 5
-7 604 3
-92 53 3
-328 135 3
-387 474 5
-393 748 3
-474 943 4
-268 713 4
-290 239 2
-864 128 4
-151 82 3
-291 1042 4
-551 576 2
-260 881 4
-459 323 3
-500 143 3
-843 143 2
-308 708 4
-897 479 4
-239 65 5
-92 111 3
-474 1028 1
-697 326 4
-780 318 5
-393 272 4
-581 922 5
-498 628 4
-222 180 3
-485 328 2
-209 1 5
-561 492 4
-102 302 3
-607 482 5
-795 204 3
-932 230 4
-393 27 4
-90 347 4
-711 97 4
-414 264 3
-757 91 4
-270 443 3
-880 194 5
-269 1014 3
-506 432 4
-499 205 5
-846 967 3
-311 425 2
-854 290 1
-301 772 3
-788 68 3
-782 894 2
-109 168 3
-659 762 3
-87 82 5
-322 346 3
-363 39 4
-6 480 4
-425 898 3
-600 540 3
-881 526 5
-435 3 3
-145 894 1
-313 684 4
-385 24 3
-267 579 3
-86 304 3
-58 223 5
-335 307 5
-393 953 4
-450 521 4
-655 712 3
-34 286 5
-886 202 3
-585 113 3
-709 228 3
-94 241 4
-494 191 4
-624 237 4
-280 975 4
-645 318 5
-871 1385 3
-575 96 5
-889 322 3
-694 161 4
-705 623 5
-359 50 5
-588 151 4
-764 245 4
-391 195 2
-586 742 3
-829 153 4
-142 514 5
-707 1113 2
-301 1012 4
-653 156 4
-833 30 4
-716 86 5
-627 117 3
-373 553 4
-634 515 4
-792 21 3
-188 568 4
-417 809 3
-406 485 3
-362 245 4
-822 71 4
-76 129 3
-514 189 5
-890 429 4
-373 194 4
-892 321 5
-881 205 4
-405 725 1
-650 642 3
-857 304 2
-343 188 4
-806 1018 4
-550 924 4
-25 969 3
-405 1194 1
-768 269 3
-13 451 1
-195 797 3
-277 181 3
-716 427 5
-21 406 1
-145 202 4
-405 168 1
-174 21 1
-520 893 2
-346 496 5
-535 614 5
-142 294 3
-634 341 2
-404 289 1
-711 283 4
-437 606 4
-615 187 5
-13 474 4
-174 107 5
-719 456 1
-268 1095 2
-640 210 5
-343 461 2
-716 417 3
-228 275 3
-207 685 3
-23 156 3
-490 123 2
-768 274 3
-621 241 4
-276 410 4
-642 673 2
-454 736 3
-665 687 2
-605 153 4
-178 596 3
-257 305 4
-592 919 5
-782 688 2
-524 107 3
-671 2 4
-932 163 4
-429 1285 3
-326 474 5
-870 10 4
-881 812 2
-635 300 3
-881 1089 1
-457 507 4
-63 282 1
-178 229 4
-490 100 3
-880 689 4
-537 1194 3
-299 855 4
-418 331 3
-94 727 5
-232 48 5
-85 231 2
-216 257 3
-307 143 3
-297 480 4
-426 428 2
-541 419 5
-405 461 3
-804 1050 3
-576 7 5
-562 79 4
-399 56 3
-851 95 4
-629 210 5
-92 278 3
-465 511 4
-761 243 3
-921 720 4
-807 627 4
-653 4 3
-230 228 2
-387 193 5
-294 826 1
-708 1040 2
-378 356 4
-385 653 4
-682 325 4
-675 891 2
-297 628 4
-705 8 3
-864 1531 3
-577 582 4
-795 568 3
-823 1135 3
-368 217 5
-913 9 5
-712 796 4
-267 614 5
-372 443 4
-885 946 3
-618 203 3
-370 603 5
-838 71 3
-514 50 5
-495 496 5
-561 474 5
-457 164 4
-660 542 2
-601 416 3
-751 204 4
-376 154 4
-425 231 3
-13 829 3
-299 959 2
-586 436 2
-141 1013 1
-663 1 4
-624 275 4
-655 451 3
-758 576 4
-138 518 4
-291 1079 2
-328 939 4
-186 250 1
-881 831 2
-804 412 2
-365 591 4
-279 101 3
-443 343 5
-363 288 4
-640 91 4
-472 1074 5
-13 785 3
-606 1065 5
-886 423 3
-250 1199 3
-608 694 3
-854 1677 3
-882 739 4
-699 591 2
-1 194 4
-345 742 4
-391 435 5
-458 736 4
-711 684 3
-835 162 5
-92 575 2
-886 715 1
-346 54 4
-442 181 4
-557 739 3
-776 510 5
-865 99 1
-210 357 5
-340 423 4
-491 236 4
-668 82 4
-454 875 1
-255 98 5
-97 919 5
-647 257 2
-311 197 4
-295 794 4
-279 1288 4
-921 411 2
-771 69 5
-299 23 4
-90 1203 5
-83 110 4
-788 1478 3
-664 180 4
-363 679 4
-682 410 3
-216 82 4
-699 116 4
-299 1223 3
-234 204 2
-903 183 4
-102 99 2
-524 222 2
-158 107 3
-886 98 4
-179 331 2
-862 60 5
-54 325 3
-354 181 4
-537 15 3
-262 195 2
-417 710 4
-198 462 3
-804 7 4
-102 222 3
-544 313 5
-250 98 5
-535 8 4
-130 98 5
-757 64 5
-690 655 4
-455 627 3
-846 172 4
-393 755 3
-630 412 1
-913 222 3
-466 161 2
-90 639 5
-838 181 5
-251 595 3
-769 546 4
-650 1419 3
-922 99 4
-474 55 4
-880 288 4
-397 435 4
-416 915 5
-585 70 5
-796 785 5
-398 72 3
-128 815 3
-932 121 3
-880 685 4
-435 401 3
-307 200 3
-870 490 3
-425 379 2
-92 157 4
-276 449 2
-268 824 2
-342 88 1
-197 568 4
-56 655 4
-491 124 5
-536 195 4
-711 417 4
-455 318 3
-588 1058 2
-495 44 3
-41 100 4
-393 1446 5
-641 23 5
-778 117 3
-227 7 5
-201 475 4
-609 147 1
-159 872 1
-642 13 4
-138 617 4
-485 346 4
-290 651 3
-291 15 5
-730 15 4
-327 31 2
-276 33 4
-561 193 3
-64 64 4
-932 199 5
-296 274 4
-840 517 4
-352 210 3
-618 192 5
-332 1042 4
-94 525 5
-307 1028 4
-807 265 5
-788 726 4
-686 208 5
-95 712 2
-452 265 3
-726 832 5
-249 161 3
-297 271 2
-618 421 3
-311 88 4
-24 180 5
-432 1049 2
-627 690 5
-561 343 4
-780 172 5
-85 161 4
-405 540 1
-280 7 4
-454 472 3
-787 879 4
-293 16 2
-117 288 3
-528 210 5
-343 175 5
-505 191 3
-511 322 3
-354 429 3
-409 890 1
-864 69 5
-860 344 3
-200 609 3
-399 395 3
-828 269 4
-334 70 3
-279 976 3
-454 199 3
-6 497 4
-618 483 5
-880 33 3
-851 240 4
-56 386 3
-239 518 3
-116 1020 3
-63 259 3
-579 258 5
-660 209 4
-643 385 3
-320 145 4
-796 731 3
-768 65 4
-943 763 4
-650 511 5
-655 1155 3
-283 393 4
-712 584 4
-378 285 4
-692 56 3
-660 810 3
-634 325 1
-116 145 2
-6 500 4
-145 268 4
-356 315 4
-553 378 3
-912 143 5
-90 387 5
-389 604 4
-698 529 5
-299 432 3
-880 468 3
-429 193 4
-715 955 4
-763 286 4
-181 879 2
-715 546 4
-486 285 5
-445 181 2
-110 238 3
-927 405 5
-31 504 5
-253 95 4
-385 32 5
-181 112 1
-336 1011 2
-345 284 4
-249 23 4
-354 89 4
-889 297 3
-751 597 2
-860 381 3
-504 88 3
-830 265 5
-32 240 2
-417 106 2
-889 92 3
-145 738 3
-145 1009 2
-109 9 3
-108 222 2
-389 739 2
-119 70 3
-18 482 5
-429 378 3
-569 302 4
-222 375 1
-299 480 4
-472 584 1
-653 97 3
-883 50 4
-643 87 5
-97 429 4
-435 271 4
-70 142 3
-716 628 3
-720 319 3
-405 1306 1
-344 597 2
-432 237 5
-556 243 1
-654 118 2
-472 177 4
-456 174 4
-712 415 4
-788 323 3
-49 208 4
-181 1162 1
-348 1120 3
-450 403 4
-292 58 5
-648 780 1
-213 603 5
-893 290 3
-109 385 4
-461 1006 5
-848 431 5
-62 225 3
-468 178 5
-229 311 5
-305 127 5
-741 48 4
-179 269 3
-846 417 4
-508 232 3
-552 117 3
-935 282 4
-222 185 4
-815 712 3
-101 109 2
-279 659 5
-830 431 3
-158 182 5
-932 203 4
-863 264 3
-395 1060 2
-683 306 3
-62 271 1
-450 1297 4
-109 234 4
-207 239 3
-466 265 3
-101 831 3
-470 952 3
-864 1284 3
-216 697 4
-449 212 5
-380 86 4
-647 202 4
-234 488 4
-279 727 3
-882 515 5
-342 152 4
-602 508 3
-85 1137 4
-885 50 3
-695 682 1
-269 177 5
-363 1014 1
-650 679 3
-694 692 4
-682 70 4
-405 422 1
-34 332 5
-872 928 2
-43 40 3
-429 71 3
-682 218 3
-234 207 2
-567 673 3
-70 678 3
-385 448 3
-291 17 4
-416 659 5
-162 685 3
-806 90 4
-593 763 3
-346 288 2
-59 562 4
-323 322 2
-113 292 3
-308 1147 4
-487 588 5
-288 632 4
-315 48 4
-929 515 5
-481 435 5
-899 111 4
-110 2 3
-41 188 4
-292 654 5
-162 147 4
-847 151 4
-934 212 4
-422 1199 3
-889 318 4
-418 750 2
-194 402 3
-226 283 2
-207 986 3
-119 7 5
-805 161 1
-806 628 3
-76 135 5
-305 856 5
-613 1315 4
-361 79 4
-532 840 4
-796 530 3
-880 121 2
-161 257 3
-543 102 4
-896 274 2
-187 8 5
-807 450 4
-393 354 4
-780 70 2
-405 341 1
-184 286 4
-478 77 1
-103 257 3
-312 604 5
-545 181 5
-805 65 3
-846 425 5
-798 50 5
-458 589 4
-234 1048 3
-892 210 4
-788 157 5
-145 696 3
-642 202 3
-59 65 4
-791 301 3
-882 294 4
-314 1471 4
-308 393 4
-705 282 5
-618 313 4
-711 215 3
-620 409 4
-815 159 3
-222 22 5
-11 430 3
-897 588 4
-776 438 2
-375 288 4
-271 199 4
-895 742 4
-201 654 3
-354 255 2
-843 654 2
-1 130 3
-240 748 3
-793 742 3
-870 704 3
-299 387 2
-178 846 3
-291 80 4
-871 271 5
-209 1086 4
-457 371 4
-5 364 1
-201 93 5
-90 180 4
-230 742 5
-455 97 5
-429 549 4
-815 252 2
-125 283 5
-308 44 4
-172 580 4
-587 300 4
-387 229 2
-458 199 4
-442 240 2
-236 179 1
-308 248 4
-663 326 4
-320 92 5
-308 210 4
-882 118 4
-276 340 5
-13 779 3
-224 332 3
-472 796 4
-92 38 3
-330 15 5
-555 1013 4
-632 91 3
-487 252 1
-330 172 5
-911 478 5
-806 50 5
-13 362 4
-452 76 4
-159 1037 2
-878 97 3
-479 32 3
-882 151 5
-56 216 4
-174 871 1
-299 753 5
-152 157 5
-890 452 2
-74 351 3
-693 197 3
-60 835 4
-601 96 2
-214 166 4
-892 430 5
-122 715 5
-478 975 4
-526 258 3
-561 304 3
-345 87 5
-363 1019 5
-237 185 4
-76 137 5
-45 108 4
-542 396 4
-641 303 3
-883 989 5
-724 288 4
-869 1079 2
-57 456 3
-75 273 5
-806 518 3
-627 679 3
-279 172 2
-921 419 5
-854 475 4
-852 678 3
-305 48 5
-416 257 3
-447 25 4
-110 739 4
-886 692 3
-85 971 3
-246 17 2
-344 273 4
-782 1528 2
-76 23 5
-44 665 1
-380 736 4
-383 603 5
-378 403 4
-661 501 4
-190 24 3
-116 1134 4
-294 472 3
-880 376 3
-186 1213 3
-416 199 5
-222 849 4
-641 192 4
-339 772 4
-795 1036 2
-445 208 2
-715 155 4
-718 273 3
-138 151 4
-660 17 1
-1 111 5
-749 187 3
-225 478 5
-181 829 1
-603 229 4
-55 1089 1
-864 735 5
-268 184 4
-896 436 3
-876 238 4
-748 227 3
-184 164 3
-586 83 2
-146 313 4
-83 756 4
-492 1121 2
-43 88 5
-878 1149 4
-537 288 2
-834 272 4
-224 157 4
-211 205 5
-46 1062 5
-314 24 1
-833 160 5
-287 56 5
-220 332 3
-731 190 5
-782 352 1
-58 425 5
-807 135 5
-267 433 5
-889 268 4
-158 694 5
-99 125 4
-60 222 4
-113 327 5
-586 423 2
-727 465 2
-417 800 2
-422 717 3
-776 181 4
-577 228 3
-296 1073 5
-170 322 5
-246 416 3
-531 323 5
-243 1148 3
-203 332 5
-537 322 1
-894 961 4
-666 134 5
-239 690 1
-288 289 3
-942 879 4
-665 924 4
-67 25 4
-877 271 4
-554 31 4
-862 472 5
-379 141 4
-747 176 4
-420 302 4
-531 288 1
-655 1166 3
-643 546 3
-267 568 4
-174 147 4
-858 515 4
-648 1271 4
-54 24 1
-606 735 5
-361 525 4
-397 197 5
-842 313 4
-532 842 4
-423 292 4
-425 2 2
-670 191 4
-28 11 4
-374 572 2
-559 393 2
-7 607 3
-223 819 3
-490 847 3
-890 484 3
-401 477 1
-197 403 3
-450 215 5
-153 678 2
-460 515 5
-666 116 4
-262 754 3
-940 96 5
-893 323 2
-940 183 3
-208 211 5
-934 25 4
-532 354 4
-333 483 4
-158 483 5
-402 117 3
-43 73 4
-82 183 3
-654 1165 1
-15 889 3
-542 531 4
-64 239 3
-429 491 3
-291 421 4
-378 326 3
-741 209 3
-457 304 4
-312 165 5
-276 993 3
-13 216 3
-7 179 5
-457 729 4
-147 269 4
-184 40 4
-618 64 4
-554 1042 3
-553 609 4
-260 319 2
-463 304 3
-87 405 4
-548 659 4
-435 392 3
-77 498 5
-862 520 4
-57 833 4
-474 486 4
-831 197 4
-605 295 4
-865 546 1
-297 128 4
-853 748 2
-905 751 3
-65 365 3
-561 223 4
-506 54 4
-631 346 4
-286 709 4
-717 281 4
-486 222 3
-416 56 5
-21 978 1
-486 591 4
-144 508 4
-702 271 1
-1 17 3
-881 22 5
-919 11 4
-151 561 3
-224 276 3
-870 98 4
-299 811 4
-472 928 4
-892 102 3
-48 603 4
-59 147 5
-438 476 5
-381 898 5
-577 427 4
-862 678 4
-868 183 5
-668 286 4
-561 235 3
-826 588 4
-450 1480 3
-774 235 1
-182 191 4
-409 97 5
-495 479 4
-825 130 2
-248 1 3
-721 681 3
-393 739 3
-577 118 3
-652 323 3
-363 1052 3
-776 318 4
-846 672 4
-429 147 2
-499 56 4
-655 1368 5
-749 292 4
-257 61 5
-429 1101 5
-323 249 3
-24 178 5
-99 313 5
-13 172 5
-94 181 4
-228 313 5
-342 160 3
-259 15 3
-802 672 3
-551 188 5
-758 338 4
-158 744 4
-5 410 1
-123 511 5
-551 399 3
-625 655 3
-643 132 5
-659 1119 4
-344 716 3
-592 730 4
-244 204 4
-897 290 4
-605 294 4
-49 1036 2
-326 399 4
-174 417 4
-54 121 4
-378 629 5
-59 58 4
-234 1198 3
-928 135 4
-534 763 4
-91 423 5
-10 283 4
-625 484 4
-698 176 4
-503 216 5
-705 94 4
-200 205 4
-181 1057 2
-936 272 4
-682 238 3
-308 170 3
-554 204 5
-806 121 4
-311 748 4
-85 1121 3
-385 169 5
-355 310 4
-178 655 4
-504 300 4
-796 265 5
-605 118 3
-730 327 2
-406 1079 2
-236 223 5
-94 232 3
-682 1035 3
-669 664 4
-95 83 5
-60 609 3
-535 58 5
-682 49 3
-830 127 4
-707 747 3
-312 132 5
-903 709 4
-640 201 4
-698 421 2
-843 154 3
-293 485 3
-650 186 4
-117 117 5
-253 125 3
-712 568 5
-69 151 5
-533 88 4
-14 121 3
-311 614 4
-170 304 4
-875 421 4
-307 1140 2
-600 515 5
-907 724 5
-69 1142 4
-796 781 4
-181 337 1
-130 465 5
-766 1021 2
-671 597 4
-1 89 5
-298 496 5
-1 68 4
-301 182 5
-766 132 4
-551 156 5
-710 174 4
-498 269 4
-6 154 3
-563 50 5
-503 237 4
-43 385 5
-624 473 3
-347 223 4
-840 100 5
-107 313 2
-748 318 5
-468 258 4
-393 625 4
-821 281 3
-733 13 3
-788 356 4
-764 531 5
-393 29 4
-526 245 2
-807 177 4
-656 327 2
-497 2 1
-376 11 4
-758 479 5
-839 244 3
-342 137 2
-699 70 4
-569 300 3
-183 1215 1
-177 334 3
-901 1120 4
-332 824 3
-682 259 3
-435 135 3
-293 401 1
-500 133 3
-592 752 4
-815 402 5
-59 476 2
-586 405 5
-416 364 2
-73 748 2
-298 333 5
-929 28 4
-85 566 3
-440 749 3
-472 826 3
-746 265 4
-426 133 5
-11 455 3
-15 283 4
-295 747 4
-610 276 4
-269 423 4
-747 192 5
-642 1079 5
-328 665 2
-94 820 1
-280 1048 4
-334 657 4
-342 89 3
-826 771 3
-7 544 3
-25 183 4
-747 93 4
-222 405 3
-13 431 1
-95 1228 3
-49 172 1
-38 1029 1
-305 97 4
-130 118 4
-724 326 4
-715 655 4
-130 358 4
-933 523 4
-450 1286 3
-561 1220 2
-59 926 1
-195 258 4
-472 771 4
-896 1220 1
-747 631 5
-566 235 3
-922 176 3
-919 358 3
-804 558 3
-429 404 4
-354 186 4
-130 305 4
-825 491 4
-389 77 2
-373 131 4
-458 1048 4
-815 217 3
-567 521 3
-154 324 2
-335 347 5
-18 496 5
-450 241 4
-405 1230 1
-766 837 3
-934 1018 4
-812 1393 3
-682 204 3
-525 322 2
-690 204 3
-145 1210 1
-222 281 3
-378 95 4
-527 181 4
-233 923 4
-896 471 3
-500 735 4
-18 189 5
-619 174 4
-312 28 4
-915 328 2
-524 12 3
-890 489 4
-434 1051 3
-292 607 4
-97 197 3
-75 290 4
-363 227 4
-862 276 5
-504 276 3
-167 1147 4
-92 64 4
-321 216 4
-280 472 2
-298 204 4
-835 179 5
-430 436 4
-753 64 4
-545 174 4
-18 94 3
-130 779 4
-267 17 4
-478 178 4
-119 82 2
-278 286 5
-4 294 5
-896 222 4
-493 25 4
-711 70 5
-805 405 3
-215 238 2
-421 4 3
-343 100 5
-98 517 5
-329 282 3
-49 1009 3
-405 92 1
-345 262 5
-64 187 5
-794 269 5
-345 702 4
-555 168 4
-450 794 5
-326 633 4
-83 371 3
-391 652 4
-402 237 4
-727 433 5
-188 234 4
-292 235 3
-445 221 1
-233 135 4
-864 161 4
-846 837 5
-886 177 4
-184 66 4
-308 211 4
-378 508 4
-435 42 3
-92 1018 4
-13 832 4
-474 69 5
-392 312 4
-601 418 2
-774 367 2
-823 153 4
-126 682 1
-266 325 1
-474 503 4
-642 1023 3
-833 483 4
-933 1 3
-130 94 5
-254 622 4
-200 58 4
-54 313 4
-834 294 3
-325 180 4
-786 228 4
-181 1202 1
-472 88 2
-87 849 5
-618 485 3
-317 323 2
-711 162 5
-280 77 3
-938 276 3
-556 172 5
-393 789 1
-85 512 3
-908 98 5
-426 197 4
-112 301 3
-772 264 4
-258 313 5
-883 312 3
-553 45 4
-758 123 1
-318 47 2
-246 67 2
-697 369 5
-537 654 3
-180 785 4
-851 717 3
-345 462 5
-533 203 4
-95 627 4
-524 382 3
-190 302 5
-865 240 2
-350 340 4
-171 354 3
-230 515 5
-393 1228 3
-652 275 4
-592 191 5
-363 265 3
-437 28 3
-87 657 4
-557 337 5
-655 647 3
-360 483 5
-514 715 4
-684 73 4
-883 168 5
-174 143 5
-870 1046 3
-13 9 3
-689 410 1
-303 364 2
-469 603 5
-318 660 3
-796 549 3
-943 233 5
-537 25 2
-834 127 5
-864 568 4
-501 342 4
-663 323 2
-734 751 4
-453 168 4
-44 378 3
-843 627 2
-405 583 1
-276 206 5
-378 100 4
-582 1215 4
-305 427 5
-566 192 5
-327 568 2
-620 769 4
-854 357 4
-180 658 5
-880 1664 4
-914 155 5
-271 485 4
-506 38 3
-528 83 5
-342 123 5
-378 1168 3
-610 127 5
-454 526 4
-221 1090 3
-851 1314 1
-145 924 2
-506 660 3
-474 215 5
-279 364 4
-280 176 3
-545 419 3
-528 56 3
-561 144 3
-197 227 3
-293 135 5
-439 246 4
-658 515 5
-244 871 3
-789 127 5
-312 675 5
-561 980 3
-592 269 4
-601 71 1
-43 127 4
-711 51 4
-450 582 4
-349 325 3
-398 203 4
-90 971 4
-698 83 5
-405 568 4
-387 1012 4
-141 815 4
-236 427 5
-523 384 3
-381 129 4
-940 321 4
-44 591 4
-276 334 4
-521 22 4
-501 546 4
-271 644 3
-373 184 4
-44 95 4
-342 950 2
-435 780 2
-189 136 4
-645 447 3
-62 568 3
-360 56 4
-326 616 5
-843 380 3
-221 1218 3
-476 201 4
-389 474 5
-486 299 1
-796 31 4
-7 193 5
-245 133 2
-79 283 4
-495 379 5
-871 127 5
-116 758 1
-543 381 4
-822 189 4
-421 183 5
-790 97 2
-746 83 4
-429 1112 3
-321 199 4
-943 100 5
-712 729 5
-708 473 1
-64 168 5
-118 135 5
-200 768 4
-7 7 5
-18 493 5
-87 692 5
-450 487 4
-249 597 2
-804 141 3
-757 1016 3
-279 1228 4
-655 210 3
-825 1291 2
-21 925 2
-805 142 4
-13 60 4
-870 111 3
-280 90 4
-676 13 1
-538 11 4
-577 313 4
-254 94 3
-837 258 4
-720 898 4
-906 408 4
-3 272 2
-789 181 4
-321 615 5
-417 946 4
-64 516 5
-327 405 2
-311 310 4
-357 334 4
-457 210 5
-682 186 4
-13 243 3
-455 95 4
-892 239 4
-72 147 5
-687 678 4
-807 71 5
-667 962 2
-399 450 2
-365 340 5
-344 100 5
-726 833 5
-359 751 4
-536 449 4
-707 923 5
-717 815 3
-711 393 4
-208 208 4
-223 298 5
-647 631 4
-50 823 3
-393 50 5
-314 501 4
-819 1537 5
-503 8 5
-524 467 4
-178 319 1
-363 698 2
-553 238 5
-442 98 4
-863 872 2
-918 381 5
-830 230 3
-807 657 4
-846 268 4
-554 111 4
-181 508 3
-450 604 4
-435 123 2
-904 278 5
-489 272 5
-239 671 5
-341 887 5
-755 264 2
-788 56 3
-848 202 5
-347 22 5
-89 1048 3
-234 193 4
-588 202 1
-937 268 1
-741 423 3
-435 444 3
-880 986 3
-543 174 4
-211 423 5
-184 50 4
-705 849 3
-795 402 2
-91 192 4
-218 516 5
-180 1131 5
-405 1207 1
-393 84 3
-279 1070 3
-479 535 3
-764 86 3
-354 162 3
-222 424 1
-920 300 3
-409 854 4
-109 121 5
-243 306 4
-846 1206 3
-650 301 2
-653 219 1
-551 80 1
-83 63 4
-788 82 3
-603 222 4
-38 1037 4
-664 69 3
-600 1419 3
-215 202 4
-855 529 4
-916 186 3
-870 313 4
-269 69 1
-833 654 5
-486 813 5
-478 591 3
-295 86 5
-567 475 4
-99 1016 5
-15 291 3
-17 628 1
-314 1063 5
-732 304 5
-346 1228 4
-157 340 5
-485 319 3
-311 322 4
-642 35 2
-694 659 4
-84 815 4
-269 792 4
-907 198 5
-868 651 5
-77 154 5
-657 269 5
-489 360 5
-349 118 2
-465 97 2
-274 713 5
-223 11 3
-514 137 3
-497 645 3
-831 181 5
-316 735 4
-458 195 4
-316 98 5
-48 71 3
-933 940 1
-506 1244 2
-907 272 5
-870 649 4
-913 408 5
-450 142 5
-83 105 2
-379 576 4
-630 1197 3
-537 79 3
-72 528 4
-881 323 2
-290 683 2
-393 380 2
-601 87 4
-95 433 4
-727 265 4
-129 990 2
-727 201 4
-232 181 4
-929 271 2
-47 303 4
-405 53 2
-837 9 3
-195 47 5
-359 405 3
-288 176 4
-235 494 4
-593 69 5
-645 184 3
-875 176 4
-711 169 5
-49 235 2
-557 246 5
-748 4 4
-360 144 2
-151 425 4
-57 282 5
-287 294 5
-64 98 4
-768 16 3
-756 89 4
-479 831 2
-652 699 5
-788 188 4
-655 935 3
-303 953 3
-378 727 4
-342 93 4
-545 232 3
-890 524 4
-151 228 5
-393 402 3
-556 127 5
-249 100 5
-736 678 1
-621 578 5
-693 632 5
-844 403 3
-82 326 2
-308 741 4
-274 117 4
-276 158 3
-145 789 4
-794 557 4
-378 281 3
-417 631 3
-372 872 4
-416 125 5
-707 490 2
-468 71 5
-747 1020 4
-343 53 5
-7 669 1
-59 209 5
-825 307 4
-897 419 4
-391 59 5
-219 906 4
-495 1188 5
-524 32 4
-514 1160 4
-509 687 1
-122 1044 5
-95 976 2
-75 1048 4
-234 280 3
-504 620 4
-539 215 4
-541 393 3
-682 628 4
-444 678 3
-192 340 4
-851 1025 2
-381 142 3
-647 134 4
-178 70 4
-535 645 4
-92 1041 3
-64 284 4
-383 135 5
-38 328 4
-620 138 5
-560 109 3
-271 117 3
-406 215 3
-545 205 4
-618 161 4
-198 658 3
-871 435 3
-249 431 5
-224 423 4
-653 756 1
-749 214 3
-183 265 2
-747 108 4
-312 178 5
-721 1221 3
-749 159 4
-846 421 4
-327 746 3
-639 615 5
-381 20 5
-13 554 2
-804 363 4
-156 86 4
-627 693 2
-85 630 3
-883 748 5
-497 765 3
-397 680 1
-52 111 4
-805 472 2
-10 144 4
-586 541 3
-325 614 4
-661 191 4
-783 895 4
-630 181 3
-218 431 3
-871 1431 4
-247 751 3
-883 96 4
-894 909 3
-95 450 2
-509 705 4
-280 715 2
-290 1013 2
-92 834 1
-102 173 3
-625 961 4
-173 303 5
-878 659 4
-96 519 4
-181 1120 1
-733 293 4
-185 205 3
-321 463 3
-486 544 4
-318 85 3
-928 98 5
-857 259 4
-334 328 3
-213 294 3
-457 12 5
-224 544 1
-854 799 3
-595 748 2
-125 80 4
-524 605 1
-424 1346 4
-500 118 3
-43 300 5
-387 559 3
-798 926 4
-361 934 3
-246 748 1
-895 988 3
-300 456 4
-615 475 4
-5 101 5
-665 248 4
-702 688 1
-308 180 5
-548 603 5
-200 176 5
-889 180 4
-264 1474 2
-450 51 4
-650 552 4
-663 1051 3
-148 194 5
-581 181 3
-774 97 2
-429 1220 3
-660 313 4
-747 428 3
-130 684 5
-208 310 4
-391 173 4
-405 1192 1
-346 1217 4
-379 79 5
-899 186 4
-201 380 1
-299 916 3
-751 849 2
-215 185 4
-429 98 4
-593 71 4
-87 153 5
-268 182 4
-279 10 4
-544 343 2
-405 180 3
-393 17 1
-543 636 3
-65 378 5
-339 404 4
-313 226 4
-899 66 4
-305 793 5
-401 275 4
-747 693 5
-351 880 2
-497 118 4
-804 28 4
-454 378 3
-33 271 4
-552 14 4
-825 117 5
-571 47 3
-479 97 3
-875 185 4
-186 1399 2
-22 294 1
-313 516 4
-587 879 1
-852 264 3
-510 881 2
-610 751 4
-69 9 4
-798 821 5
-847 157 1
-438 245 5
-659 316 4
-363 448 5
-551 550 5
-727 386 2
-650 1118 3
-665 238 4
-484 197 4
-896 575 2
-826 385 5
-618 233 3
-345 651 4
-474 945 4
-575 603 5
-435 222 3
-264 430 5
-193 1078 4
-795 238 3
-634 546 4
-158 275 5
-301 62 3
-345 285 5
-592 69 5
-370 257 5
-774 520 3
-705 193 3
-287 246 4
-92 582 5
-194 780 2
-176 874 4
-60 131 4
-416 501 5
-922 229 4
-339 1039 4
-755 875 1
-831 877 2
-569 258 5
-654 1009 3
-319 350 3
-328 149 2
-84 385 4
-67 756 3
-478 354 3
-294 752 3
-374 156 2
-75 114 4
-606 717 3
-275 501 3
-648 743 1
-234 1454 3
-801 333 5
-308 73 3
-393 623 3
-398 602 4
-653 83 5
-559 660 1
-778 423 1
-435 409 3
-740 271 2
-932 529 4
-445 408 3
-847 172 4
-64 203 4
-455 620 3
-345 298 5
-748 192 3
-454 322 2
-416 287 4
-352 82 3
-848 89 5
-425 424 2
-7 595 2
-650 393 3
-442 569 2
-882 21 2
-683 258 3
-830 241 4
-497 300 3
-417 40 3
-92 1194 4
-889 493 3
-179 354 4
-405 735 5
-418 269 5
-312 83 4
-417 732 4
-14 42 4
-622 756 3
-271 51 4
-588 552 1
-401 191 4
-567 12 4
-373 290 5
-894 690 4
-771 1 5
-56 7 5
-918 638 4
-280 403 3
-633 654 3
-823 475 5
-699 324 4
-364 288 4
-626 681 1
-557 268 5
-164 845 3
-475 313 2
-709 232 5
-303 238 4
-642 80 5
-497 363 2
-580 294 4
-864 775 1
-346 167 2
-268 449 2
-648 405 4
-738 204 4
-145 683 3
-271 126 3
-514 1 5
-269 806 3
-618 404 5
-293 765 3
-910 210 4
-512 527 5
-311 141 4
-337 742 5
-391 100 4
-392 510 4
-268 62 3
-405 782 1
-389 346 4
-236 274 1
-537 687 1
-308 255 4
-655 576 2
-171 245 3
-883 338 4
-894 111 3
-95 462 4
-339 518 5
-44 90 2
-886 20 2
-192 277 3
-936 358 4
-389 301 4
-892 81 3
-563 403 4
-94 356 4
-468 498 5
-329 186 3
-606 55 4
-286 884 5
-7 523 4
-271 275 4
-97 97 5
-303 187 5
-387 114 5
-758 430 5
-264 4 4
-343 98 5
-495 507 4
-447 405 2
-540 508 4
-864 282 3
-347 79 5
-723 9 3
-447 737 4
-1 266 1
-495 109 5
-621 143 2
-627 810 3
-457 758 2
-768 50 4
-174 315 5
-405 60 1
-852 289 2
-632 651 5
-815 1133 3
-303 455 3
-299 72 3
-591 306 5
-864 93 3
-16 156 4
-868 1035 1
-43 254 3
-815 214 5
-846 794 5
-924 605 3
-372 581 5
-374 79 4
-308 254 2
-830 172 5
-826 403 4
-887 596 5
-782 748 4
-682 380 4
-399 655 3
-551 147 4
-742 127 5
-655 731 3
-343 12 5
-521 651 3
-433 60 5
-577 708 3
-87 80 4
-536 2 4
-833 289 1
-144 181 4
-135 228 4
-474 510 4
-479 180 4
-201 324 5
-7 636 4
-532 554 4
-16 100 5
-303 562 4
-497 249 5
-435 1109 3
-293 156 4
-363 431 2
-305 195 3
-21 222 2
-709 1218 4
-860 846 2
-263 662 4
-454 610 3
-339 53 4
-423 148 3
-129 995 2
-201 357 4
-254 286 1
-13 448 1
-39 307 2
-567 582 3
-878 258 3
-882 89 5
-406 1170 4
-163 97 4
-894 268 3
-627 399 3
-59 367 4
-570 886 2
-416 148 5
-515 895 4
-924 276 2
-255 834 4
-437 91 3
-666 636 4
-109 147 4
-313 415 2
-198 118 2
-151 183 3
-241 689 3
-788 23 3
-31 135 4
-116 607 2
-168 866 5
-618 241 4
-207 265 3
-653 310 4
-715 471 4
-682 716 2
-560 472 2
-695 288 4
-399 43 3
-269 1017 5
-10 602 5
-314 546 4
-261 1237 3
-43 516 5
-747 25 3
-128 684 4
-738 214 4
-851 291 4
-504 1147 4
-438 815 5
-102 667 3
-452 102 2
-108 121 3
-619 226 5
-451 1392 1
-331 414 4
-150 93 4
-291 106 4
-63 1008 3
-731 485 4
-561 794 2
-297 213 3
-327 191 4
-26 181 4
-518 864 3
-893 411 3
-804 39 2
-246 735 4
-95 398 1
-320 771 3
-711 710 4
-248 7 2
-712 51 3
-276 1031 2
-816 323 4
-379 345 3
-374 193 4
-632 7 3
-741 181 4
-669 150 3
-610 97 3
-58 116 5
-804 237 4
-653 441 3
-267 515 5
-291 800 2
-782 302 3
-159 832 3
-383 340 5
-790 62 3
-566 265 4
-437 655 4
-326 205 4
-659 134 4
-87 685 3
-839 1245 4
-606 326 4
-268 562 4
-493 274 5
-233 644 5
-744 9 3
-174 28 5
-476 579 2
-244 553 5
-457 238 5
-373 24 4
-880 768 2
-535 504 3
-186 55 4
-711 49 4
-275 449 3
-6 537 4
-158 226 3
-430 56 4
-457 699 4
-658 24 3
-487 823 1
-389 79 4
-117 174 4
-796 155 5
-405 1575 1
-433 174 5
-361 513 5
-222 1291 2
-252 742 4
-804 1170 3
-145 770 1
-248 198 5
-592 326 4
-64 271 3
-285 205 4
-536 168 5
-255 826 1
-232 461 5
-262 473 2
-757 1 4
-749 527 4
-417 23 3
-709 403 3
-303 85 3
-416 747 5
-533 122 1
-717 274 4
-437 66 3
-843 179 4
-846 210 5
-864 474 4
-502 294 3
-588 354 5
-279 1025 2
-130 176 5
-6 513 4
-194 756 1
-577 1147 4
-645 367 3
-452 97 4
-151 408 5
-181 243 1
-183 483 5
-363 316 3
-655 1395 3
-627 76 3
-22 118 4
-381 493 4
-200 982 2
-532 562 5
-514 342 1
-424 100 5
-386 825 4
-618 125 3
-128 942 5
-163 433 1
-33 895 3
-279 464 4
-115 176 5
-418 328 1
-391 474 5
-17 126 4
-59 100 5
-59 484 4
-269 1073 3
-378 73 3
-320 410 4
-116 760 3
-868 747 2
-720 268 4
-585 1005 4
-393 779 3
-592 1048 3
-426 614 4
-63 106 2
-435 24 4
-54 268 5
-776 551 3
-674 678 3
-880 570 3
-32 288 4
-213 582 4
-181 410 1
-116 914 2
-763 5 4
-637 926 2
-897 389 3
-458 423 2
-598 286 5
-255 827 2
-373 423 2
-241 350 2
-838 354 4
-405 1208 1
-49 1018 2
-904 682 4
-234 607 4
-387 31 3
-407 869 3
-13 432 4
-927 421 4
-32 294 3
-308 487 4
-600 541 1
-344 619 4
-25 134 4
-263 465 4
-516 50 5
-197 510 5
-269 133 3
-579 331 3
-466 87 3
-627 161 2
-605 879 3
-91 300 4
-416 54 5
-504 258 5
-738 206 3
-597 763 4
-173 678 3
-567 234 3
-25 633 4
-43 175 2
-373 487 4
-308 402 4
-624 288 4
-210 393 3
-503 210 5
-308 472 2
-234 470 2
-223 749 4
-676 471 3
-875 527 4
-663 89 4
-434 975 5
-125 28 4
-480 511 4
-451 358 1
-42 1044 4
-373 150 4
-156 9 4
-399 975 2
-44 195 5
-653 282 3
-429 73 3
-487 49 4
-932 56 4
-87 238 3
-303 452 2
-268 257 4
-867 228 5
-81 147 4
-922 29 3
-862 117 5
-457 111 3
-936 827 2
-194 527 4
-788 427 2
-707 387 4
-889 483 4
-537 24 1
-305 239 3
-286 301 5
-466 232 4
-868 135 5
-339 194 4
-442 27 2
-687 245 3
-642 418 5
-608 303 4
-705 566 4
-831 258 2
-389 197 5
-18 516 5
-676 300 4
-181 1011 1
-862 168 4
-41 516 5
-896 83 5
-92 200 3
-157 748 2
-464 288 4
-7 561 4
-653 823 2
-222 380 4
-346 68 3
-342 428 5
-293 931 1
-913 483 3
-194 144 4
-705 255 5
-150 123 4
-44 871 3
-801 881 3
-318 140 4
-234 136 4
-650 496 4
-439 300 4
-774 293 1
-655 529 4
-334 127 4
-293 770 3
-665 111 4
-268 168 4
-64 269 5
-653 802 2
-263 1020 3
-279 375 1
-512 313 3
-495 168 5
-811 258 5
-655 979 3
-535 527 3
-251 685 4
-378 79 4
-497 268 4
-889 219 2
-423 887 5
-521 250 3
-846 191 5
-546 569 4
-299 181 3
-650 188 3
-429 321 3
-586 144 4
-863 682 3
-392 300 2
-655 337 2
-417 810 3
-890 604 5
-940 316 4
-659 69 3
-551 76 4
-91 135 4
-13 576 3
-145 410 4
-270 741 5
-18 610 4
-51 50 5
-119 458 5
-496 496 1
-249 209 5
-432 276 4
-305 135 3
-234 21 3
-478 72 1
-256 1289 4
-571 174 4
-488 705 4
-883 732 3
-234 837 3
-122 519 4
-518 240 1
-622 977 2
-618 98 5
-312 234 5
-455 939 4
-777 216 4
-777 9 5
-919 946 4
-118 654 5
-697 122 4
-537 421 2
-796 198 4
-254 188 3
-775 313 4
-235 483 5
-175 193 4
-839 321 1
-193 403 3
-116 191 4
-763 627 3
-605 252 4
-10 467 4
-867 182 4
-458 79 5
-165 318 5
-833 340 5
-403 100 5
-13 12 5
-328 708 2
-181 329 1
-234 166 5
-865 475 4
-206 288 5
-798 105 3
-532 1426 3
-280 237 3
-339 1267 3
-567 479 5
-7 265 5
-943 546 4
-890 480 5
-90 1192 5
-92 679 4
-587 875 1
-254 211 3
-716 274 5
-235 429 4
-359 323 3
-929 484 3
-299 950 2
-840 705 4
-456 696 3
-339 302 4
-705 29 5
-435 732 4
-308 515 3
-334 155 2
-256 765 4
-916 241 4
-894 100 4
-15 754 5
-249 258 5
-92 783 3
-372 12 4
-90 660 4
-474 126 4
-295 727 5
-6 223 4
-233 57 5
-437 447 4
-345 1117 4
-447 845 3
-301 546 4
-839 93 4
-49 403 3
-454 606 2
-621 876 2
-606 969 5
-693 313 5
-795 42 3
-62 204 3
-699 98 4
-879 222 4
-760 111 4
-207 1023 3
-407 154 5
-299 197 3
-22 780 1
-634 340 4
-560 288 4
-323 651 5
-316 192 1
-806 100 4
-299 208 4
-625 192 2
-918 664 4
-271 511 5
-315 156 5
-222 181 4
-624 952 3
-510 326 4
-624 1114 4
-387 184 3
-705 265 5
-62 47 4
-523 1 5
-92 800 3
-95 24 3
-616 313 5
-298 125 3
-311 132 4
-154 174 5
-313 357 5
-593 660 5
-284 302 4
-655 956 3
-697 287 4
-405 566 1
-407 201 4
-270 566 5
-405 1382 1
-828 224 3
-674 763 5
-714 3 5
-249 1012 3
-705 82 5
-643 447 4
-119 299 4
-894 324 3
-207 328 2
-267 840 4
-275 520 4
-716 229 3
-916 174 5
-825 472 5
-25 1 5
-686 204 4
-842 362 3
-48 193 2
-435 117 3
-880 1134 5
-422 294 3
-456 98 3
-393 778 3
-299 1073 4
-535 210 5
-276 204 5
-308 742 4
-181 546 2
-933 483 4
-391 530 5
-655 740 3
-487 318 3
-516 199 3
-804 182 4
-589 268 1
-397 313 4
-539 45 4
-20 252 4
-268 88 2
-606 928 4
-204 340 5
-226 191 4
-268 73 3
-214 752 2
-447 69 4
-405 32 1
-13 5 1
-474 530 5
-347 215 4
-621 53 4
-638 29 2
-758 152 5
-896 880 4
-267 642 4
-63 841 1
-215 483 4
-13 523 4
-95 739 3
-354 655 3
-445 288 2
-342 124 4
-18 9 5
-435 10 5
-204 269 4
-650 636 3
-830 511 5
-104 508 2
-269 108 5
-354 694 5
-606 22 5
-561 503 4
-246 92 1
-198 196 3
-647 1063 3
-907 520 5
-807 1091 3
-48 172 5
-318 376 3
-880 209 3
-905 150 4
-90 42 4
-609 948 1
-601 934 1
-45 100 5
-268 144 4
-151 953 5
-780 28 5
-586 288 4
-599 748 4
-501 181 4
-234 88 3
-899 546 2
-538 483 5
-540 7 4
-826 651 4
-25 404 3
-793 222 3
-64 919 4
-193 815 3
-684 216 3
-246 50 5
-189 129 3
-870 79 4
-455 300 4
-93 15 5
-670 519 5
-830 161 4
-881 304 3
-193 755 4
-334 293 3
-650 485 3
-13 756 2
-406 1065 2
-310 740 4
-244 240 3
-429 1020 4
-121 427 4
-654 742 4
-661 179 4
-388 98 5
-5 183 4
-334 235 3
-315 46 4
-655 22 2
-40 270 3
-627 82 4
-579 228 3
-99 338 4
-181 1242 1
-826 576 4
-464 333 4
-645 474 5
-174 9 5
-397 7 5
-810 338 4
-603 474 4
-561 640 5
-178 235 1
-339 475 5
-436 204 5
-305 89 3
-424 14 4
-276 145 3
-489 875 2
-114 507 3
-158 117 3
-5 214 3
-173 995 5
-632 174 5
-595 922 4
-201 375 3
-44 194 5
-82 87 3
-405 1587 1
-110 780 3
-25 238 4
-617 444 4
-886 181 5
-184 458 3
-43 313 5
-436 219 5
-343 744 4
-393 62 4
-13 187 5
-65 197 5
-892 478 5
-892 481 5
-717 471 4
-899 144 3
-389 194 4
-175 172 5
-916 866 3
-407 659 5
-370 52 4
-389 38 2
-474 405 4
-281 989 2
-654 154 3
-416 323 3
-344 176 5
-489 1025 5
-119 210 5
-280 977 3
-934 965 4
-280 1114 4
-684 924 2
-429 762 4
-922 77 4
-751 90 3
-20 15 4
-731 64 5
-405 1178 1
-275 304 3
-790 227 3
-645 48 4
-342 547 5
-881 89 4
-382 23 5
-131 100 5
-603 449 4
-59 276 5
-326 498 5
-168 259 2
-378 56 4
-537 491 4
-606 1277 3
-574 883 4
-85 1113 2
-488 655 3
-399 180 3
-303 151 5
-286 214 1
-186 226 5
-163 286 3
-606 427 4
-796 448 4
-830 87 4
-560 137 4
-648 185 5
-934 186 2
-758 311 4
-181 270 4
-449 593 4
-1 82 5
-848 109 4
-474 481 4
-128 965 3
-40 347 2
-235 1105 2
-268 4 4
-160 1223 4
-401 603 4
-718 546 4
-11 731 4
-323 215 5
-804 159 4
-59 519 4
-543 480 4
-508 96 2
-230 203 2
-91 515 5
-682 298 4
-474 59 3
-59 423 5
-886 55 4
-872 332 3
-83 479 5
-297 249 3
-130 420 5
-663 25 4
-21 844 4
-99 182 4
-538 98 5
-588 118 3
-357 508 5
-916 284 2
-211 117 4
-151 939 4
-633 98 4
-85 531 4
-327 651 4
-716 180 3
-445 1097 1
-588 289 2
-311 499 4
-896 651 4
-18 966 2
-526 328 2
-279 1312 3
-835 318 5
-637 460 2
-462 655 5
-592 297 5
-596 678 3
-508 173 4
-653 182 3
-405 779 1
-671 1491 1
-293 235 3
-496 318 4
-382 127 3
-13 857 3
-405 512 1
-406 772 4
-774 402 2
-58 310 4
-700 202 3
-178 184 5
-22 258 5
-653 388 2
-641 657 4
-753 673 1
-456 1220 3
-313 125 3
-590 116 5
-42 54 4
-234 550 2
-532 272 5
-686 174 4
-128 86 5
-566 230 2
-387 446 2
-854 185 4
-764 356 4
-650 662 3
-6 22 3
-615 238 3
-799 289 3
-313 309 4
-506 523 5
-714 284 3
-849 588 5
-379 151 4
-460 242 4
-670 168 3
-26 240 3
-42 111 1
-826 2 3
-472 660 5
-796 736 3
-847 763 1
-394 202 5
-758 328 1
-6 209 4
-487 249 1
-183 176 3
-331 133 3
-42 210 5
-125 236 1
-486 475 4
-838 173 5
-79 582 5
-698 187 2
-679 95 3
-210 222 4
-675 306 5
-320 257 4
-555 288 3
-882 227 4
-653 1016 3
-516 169 5
-802 264 4
-83 385 4
-543 516 4
-271 642 5
-291 685 5
-637 619 2
-235 701 4
-280 756 4
-573 205 3
-796 215 5
-841 270 4
-865 1011 1
-589 322 3
-125 190 5
-453 184 4
-407 427 4
-549 866 4
-752 905 2
-90 958 4
-321 499 3
-270 574 3
-815 871 1
-622 8 4
-416 678 2
-938 250 3
-566 423 2
-685 302 3
-436 125 4
-716 284 3
-553 213 5
-286 309 5
-310 832 1
-405 781 5
-214 117 4
-276 531 4
-724 242 1
-125 393 4
-321 497 5
-416 1428 3
-562 418 5
-670 651 4
-808 346 5
-846 493 5
-883 124 5
-318 722 4
-648 235 4
-85 488 4
-823 568 3
-206 683 1
-643 200 3
-109 89 4
-560 478 4
-195 413 3
-650 551 3
-535 25 4
-934 226 4
-325 47 3
-63 1010 3
-504 50 3
-378 419 4
-97 135 5
-264 173 5
-823 273 3
-665 24 3
-327 1103 4
-119 31 5
-13 355 3
-893 294 3
-751 659 5
-5 397 2
-543 234 4
-790 786 3
-378 866 2
-120 50 4
-43 140 4
-865 1 1
-851 1276 2
-399 276 3
-542 15 2
-184 995 3
-938 25 4
-655 863 3
-43 408 5
-442 176 5
-842 306 4
-109 1060 4
-592 876 1
-588 79 4
-386 24 4
-524 476 3
-569 281 3
-886 403 4
-7 635 3
-640 663 5
-758 185 4
-268 179 4
-450 281 4
-726 310 4
-589 301 2
-802 484 3
-194 191 4
-294 260 4
-659 708 3
-8 686 3
-658 235 2
-561 65 3
-145 105 2
-666 661 4
-286 724 3
-881 125 5
-466 33 4
-409 516 4
-387 101 4
-561 258 2
-185 447 4
-711 154 4
-505 588 5
-405 184 1
-463 116 5
-397 286 4
-846 294 3
-271 347 3
-347 99 3
-141 410 4
-294 476 3
-937 100 3
-551 49 3
-160 169 4
-744 482 3
-303 77 4
-392 495 3
-280 468 4
-835 286 3
-249 92 5
-738 56 4
-95 768 1
-735 181 4
-761 147 4
-330 575 4
-435 762 4
-889 678 3
-846 373 3
-864 511 4
-464 510 4
-862 651 5
-378 572 3
-426 478 4
-429 85 4
-524 471 4
-214 24 3
-504 575 3
-835 180 5
-348 928 5
-933 467 3
-31 153 4
-537 690 2
-807 151 4
-637 922 1
-429 629 3
-566 54 3
-399 1060 3
-49 821 1
-345 1082 2
-465 478 4
-666 300 3
-299 785 2
-932 165 4
-436 469 3
-328 427 3
-456 187 4
-894 874 4
-436 348 4
-597 15 5
-222 168 4
-561 172 2
-940 14 3
-405 173 5
-62 167 2
-536 143 5
-256 443 3
-7 484 5
-318 458 4
-109 90 3
-648 585 3
-177 12 5
-299 58 3
-117 240 3
-203 294 2
-256 147 4
-916 88 4
-59 581 5
-193 218 4
-312 486 5
-390 181 4
-728 748 3
-560 411 3
-930 237 3
-828 213 2
-258 243 3
-772 898 3
-426 493 4
-236 207 3
-774 585 1
-468 50 5
-81 1 4
-474 47 4
-830 233 3
-655 1643 5
-276 1245 3
-387 210 4
-291 11 4
-181 1358 1
-177 195 4
-326 265 4
-693 673 4
-31 1019 5
-643 55 4
-484 823 4
-478 710 5
-786 197 3
-655 543 3
-276 496 4
-788 755 3
-131 536 5
-803 306 4
-102 186 4
-307 227 5
-865 258 4
-280 385 5
-58 45 5
-439 100 3
-786 318 5
-774 672 1
-472 51 5
-721 174 5
-305 863 4
-18 411 3
-632 181 5
-562 483 4
-291 943 4
-293 26 3
-650 501 3
-887 254 4
-437 651 4
-490 273 1
-653 366 2
-514 45 4
-750 886 3
-95 176 3
-56 402 5
-642 357 2
-843 450 2
-82 946 2
-896 168 4
-521 89 3
-316 50 1
-280 584 4
-254 616 1
-659 197 5
-76 270 3
-495 1118 5
-474 601 5
-280 219 2
-625 739 3
-889 196 5
-582 258 4
-405 402 3
-216 1047 3
-795 640 4
-234 464 4
-24 55 5
-886 147 5
-709 82 4
-561 206 3
-697 751 5
-660 84 2
-58 210 4
-397 345 4
-708 121 3
-934 495 4
-677 405 4
-308 134 5
-26 150 3
-223 111 4
-486 508 4
-608 1115 4
-687 988 3
-493 50 5
-308 257 4
-267 727 4
-7 166 3
-852 926 3
-494 9 2
-745 519 5
-275 746 4
-643 1098 4
-181 1289 1
-622 173 5
-655 483 4
-521 496 2
-435 721 4
-903 595 2
-405 661 3
-937 224 4
-805 739 1
-86 888 4
-545 208 3
-216 928 3
-901 91 1
-556 134 5
-328 22 5
-167 96 5
-758 634 5
-416 568 4
-589 259 5
-563 237 5
-379 357 5
-610 294 1
-618 433 2
-276 9 5
-629 463 4
-790 328 3
-743 268 4
-416 916 3
-291 1073 5
-250 558 4
-880 161 2
-658 433 4
-450 69 4
-151 503 3
-440 1194 5
-542 214 3
-308 87 4
-756 141 3
-94 549 5
-201 747 2
-246 470 4
-82 866 3
-337 67 4
-303 38 1
-7 567 1
-332 696 3
-343 1140 3
-931 900 4
-79 124 5
-276 636 4
-374 1049 1
-184 276 4
-881 222 5
-389 378 5
-751 28 5
-883 190 4
-345 238 5
-328 432 1
-19 313 2
-223 476 3
-629 381 4
-88 886 5
-840 157 4
-123 242 5
-620 969 4
-707 286 5
-336 294 4
-389 40 3
-537 588 1
-311 684 4
-47 286 3
-880 228 3
-655 813 3
-670 228 5
-193 405 3
-637 148 3
-295 155 4
-737 192 5
-622 31 3
-429 1418 3
-436 38 3
-290 692 5
-456 460 3
-749 144 5
-782 678 3
-201 10 3
-846 601 5
-399 419 3
-260 350 4
-48 1065 2
-143 331 5
-926 322 2
-539 163 4
-489 339 3
-267 693 4
-459 22 5
-201 333 2
-607 238 4
-830 402 4
-798 258 4
-732 305 2
-569 328 4
-416 182 4
-736 748 2
-627 685 3
-561 433 1
-936 995 3
-444 286 2
-459 294 5
-532 1016 4
-619 11 2
-379 188 4
-882 121 4
-158 985 4
-787 749 4
-479 748 3
-727 67 4
-180 196 5
-837 276 1
-13 153 4
-548 164 5
-792 1054 1
-901 465 4
-498 191 4
-833 203 5
-867 1065 5
-312 207 5
-600 127 5
-405 1464 1
-541 527 3
-921 815 5
-648 619 3
-82 822 2
-102 164 3
-527 185 5
-119 323 4
-539 531 4
-85 707 4
-18 318 5
-435 171 5
-405 658 4
-401 661 3
-1 160 4
-385 160 4
-442 1183 3
-487 87 5
-407 478 4
-466 96 5
-632 550 2
-102 448 3
-84 151 4
-851 754 2
-799 127 4
-13 396 3
-451 288 5
-848 170 5
-850 168 5
-542 181 4
-498 197 5
-295 121 4
-168 291 4
-248 515 5
-577 735 5
-372 148 5
-362 323 2
-363 387 1
-686 209 5
-805 946 2
-244 763 4
-881 229 4
-780 662 5
-561 241 2
-748 189 4
-347 588 3
-271 15 3
-57 710 3
-416 273 4
-593 301 4
-573 183 3
-898 286 2
-851 1095 3
-194 404 3
-328 216 3
-514 98 5
-103 250 4
-654 239 4
-458 127 5
-506 50 5
-445 879 2
-297 750 5
-533 879 3
-85 204 4
-406 591 3
-62 275 4
-653 22 5
-514 735 4
-43 684 4
-14 663 5
-752 316 3
-488 414 2
-650 69 2
-125 117 3
-450 193 5
-196 762 3
-362 313 4
-851 284 3
-361 504 4
-435 786 4
-731 427 5
-902 483 4
-532 472 5
-896 80 2
-453 258 4
-343 260 1
-207 810 2
-331 482 2
-224 555 3
-645 50 4
-649 1283 2
-798 1283 4
-379 211 5
-269 371 5
-738 318 5
-154 89 5
-250 276 4
-201 27 3
-387 47 4
-233 204 5
-616 316 4
-374 322 4
-497 89 4
-667 192 5
-521 97 3
-864 951 3
-484 151 4
-63 323 1
-5 418 3
-315 93 5
-543 202 4
-399 400 3
-682 769 2
-682 65 3
-872 310 4
-425 195 4
-456 662 4
-506 539 4
-336 66 3
-234 739 3
-95 515 5
-682 111 3
-664 764 4
-5 394 2
-83 322 3
-697 763 4
-524 523 4
-923 455 4
-144 91 2
-727 275 3
-617 531 2
-883 16 4
-43 321 3
-533 10 2
-577 318 5
-263 141 5
-151 663 4
-65 511 4
-865 928 1
-452 729 1
-747 427 5
-239 428 5
-701 344 3
-274 591 4
-417 420 4
-33 682 4
-727 680 3
-378 543 4
-234 15 3
-422 181 4
-360 127 5
-378 1181 2
-497 95 4
-682 168 5
-82 518 4
-296 846 2
-391 460 4
-642 1063 3
-903 100 5
-684 173 3
-503 204 3
-109 82 5
-286 240 3
-99 405 4
-471 768 3
-805 83 4
-457 27 4
-840 603 5
-891 740 5
-758 356 2
-707 256 4
-932 14 4
-642 294 5
-247 222 3
-70 150 3
-345 570 2
-693 523 4
-399 684 3
-102 195 4
-463 288 1
-267 127 5
-244 723 3
-91 418 2
-291 496 5
-727 809 4
-314 775 3
-843 141 4
-457 559 4
-387 650 2
-679 196 4
-328 483 5
-246 402 3
-184 1136 4
-280 387 4
-543 71 4
-826 1239 4
-326 393 4
-462 322 5
-7 324 1
-343 189 4
-840 493 5
-344 284 3
-416 223 5
-642 969 2
-313 151 1
-380 151 4
-524 126 4
-878 707 2
-254 379 1
-486 304 3
-293 663 3
-77 127 2
-428 988 1
-73 59 5
-373 496 5
-795 8 5
-658 31 3
-497 294 4
-715 193 5
-440 171 5
-853 294 2
-210 210 5
-521 625 3
-426 174 3
-234 165 5
-559 1141 2
-651 294 1
-452 52 3
-323 246 4
-125 813 1
-624 294 3
-417 51 3
-378 302 5
-167 288 3
-290 235 3
-151 213 5
-472 401 4
-164 690 4
-719 300 2
-642 462 4
-675 427 5
-87 790 4
-514 179 4
-868 178 5
-500 1 4
-930 16 1
-630 322 3
-222 712 3
-264 709 5
-875 288 4
-590 127 4
-577 1209 4
-889 659 4
-59 68 2
-541 118 4
-264 42 5
-385 443 3
-584 228 5
-94 238 5
-606 562 4
-458 191 5
-405 1250 1
-24 153 4
-565 923 4
-374 229 5
-867 498 4
-591 580 2
-201 440 2
-530 322 4
-826 231 3
-885 625 3
-774 561 1
-648 414 1
-215 272 3
-227 240 1
-13 690 3
-768 966 4
-758 168 5
-910 250 1
-372 574 4
-804 468 4
-193 827 2
-195 386 2
-885 95 4
-409 30 4
-453 403 4
-33 872 3
-303 78 2
-457 145 3
-352 195 4
-883 1041 3
-897 135 3
-818 312 2
-786 484 4
-276 392 3
-699 978 4
-456 168 4
-806 879 3
-450 1147 4
-92 80 2
-256 121 5
-805 568 3
-343 241 3
-807 477 4
-82 225 3
-666 616 3
-430 300 3
-256 235 3
-271 197 4
-823 427 4
-503 223 5
-397 23 5
-388 895 4
-625 50 5
-374 1210 4
-710 181 3
-376 111 4
-634 934 2
-536 549 3
-702 343 2
-805 101 2
-356 300 3
-654 252 2
-373 48 5
-833 1149 4
-369 988 3
-588 732 4
-638 161 4
-282 268 4
-506 710 5
-269 474 4
-533 281 4
-121 180 3
-766 191 4
-545 566 4
-893 358 2
-354 1137 4
-735 147 1
-183 273 4
-937 301 1
-775 887 4
-92 432 3
-312 837 4
-492 69 3
-663 455 2
-130 1279 4
-561 124 3
-342 1014 1
-751 315 3
-511 682 4
-797 307 2
-787 1671 1
-624 508 4
-585 60 4
-731 720 3
-932 640 2
-577 240 3
-409 45 4
-13 751 5
-881 530 5
-870 1021 2
-347 230 4
-207 226 2
-878 371 3
-918 86 4
-894 332 3
-627 434 4
-87 403 3
-615 640 3
-16 160 4
-892 131 4
-880 245 2
-939 424 3
-416 710 4
-798 819 3
-711 197 4
-503 496 5
-532 1046 4
-626 923 5
-152 234 4
-503 633 5
-326 501 3
-642 951 3
-882 208 5
-481 514 4
-385 180 4
-871 286 3
-747 1021 5
-887 127 3
-899 255 4
-894 322 3
-138 187 5
-131 313 5
-809 300 4
-281 323 3
-457 588 5
-937 137 3
-110 195 2
-305 557 4
-292 7 3
-757 68 4
-733 151 4
-499 183 4
-303 1215 1
-437 210 3
-385 93 3
-641 513 5
-804 472 3
-870 591 2
-425 405 2
-870 2 2
-833 52 3
-188 88 4
-256 161 5
-537 714 3
-314 417 4
-201 587 4
-774 569 2
-826 190 3
-766 659 3
-429 15 5
-851 64 5
-896 966 4
-412 218 3
-806 1016 1
-232 133 4
-716 131 5
-44 9 5
-942 427 5
-349 285 5
-297 269 4
-286 704 2
-880 1217 3
-458 8 4
-416 1053 4
-724 879 1
-201 467 2
-916 14 5
-416 1054 3
-685 886 1
-682 455 4
-13 833 2
-72 520 5
-843 578 3
-452 173 4
-246 720 1
-487 222 4
-222 214 4
-307 153 5
-290 826 2
-469 705 5
-655 1195 3
-387 563 2
-650 402 3
-301 1016 4
-138 493 4
-567 50 1
-479 490 4
-653 64 4
-798 228 3
-732 269 5
-427 322 3
-118 675 5
-854 287 3
-671 298 4
-421 234 5
-18 111 3
-258 690 4
-864 526 4
-311 125 4
-757 206 4
-728 282 4
-320 546 4
-337 515 5
-90 20 4
-659 170 3
-378 528 5
-843 183 5
-619 118 5
-768 742 3
-81 591 5
-224 29 3
-608 956 3
-486 1176 3
-295 25 5
-398 414 3
-308 42 4
-237 474 5
-253 732 4
-409 538 3
-72 96 5
-1 93 5
-661 166 5
-416 845 4
-621 122 2
-59 514 5
-523 954 5
-276 450 1
-207 53 1
-298 318 5
-561 14 3
-638 554 3
-128 1053 3
-627 176 5
-864 451 4
-399 407 3
-474 175 4
-561 480 4
-407 519 4
-634 248 4
-393 1531 4
-148 507 5
-215 218 3
-704 89 5
-935 1048 3
-56 820 3
-307 1411 4
-524 173 4
-727 421 5
-379 621 4
-845 308 4
-194 1041 2
-176 881 3
-723 50 4
-543 66 3
-788 570 3
-349 237 2
-758 212 4
-901 257 4
-234 162 3
-537 6 2
-591 238 5
-666 660 4
-870 1412 2
-680 273 3
-497 928 3
-159 294 4
-13 525 5
-565 171 5
-13 200 3
-138 519 5
-70 139 3
-537 499 3
-1 22 4
-125 111 3
-262 476 3
-712 385 5
-757 53 3
-551 520 4
-22 173 5
-711 744 4
-833 223 4
-798 321 3
-405 1407 1
-339 82 4
-632 201 4
-339 327 4
-766 173 4
-748 135 4
-541 993 4
-904 785 5
-453 1037 1
-615 275 4
-308 178 4
-61 294 2
-619 144 5
-758 654 4
-270 742 2
-256 181 4
-892 56 4
-481 181 5
-587 347 3
-889 191 4
-622 185 3
-459 148 5
-334 134 5
-429 137 5
-518 121 5
-13 310 4
-353 358 1
-671 720 3
-360 936 4
-453 421 4
-682 1118 3
-846 654 5
-770 25 5
-406 218 3
-101 819 1
-858 323 2
-667 427 5
-916 290 3
-301 566 3
-749 484 5
-460 117 3
-919 676 4
-158 233 3
-747 675 2
-941 763 3
-276 95 5
-932 178 5
-514 609 4
-21 327 3
-160 514 4
-460 279 2
-492 187 5
-77 133 2
-283 186 5
-345 287 4
-667 131 5
-145 934 1
-458 474 4
-393 473 3
-445 257 2
-251 1016 3
-234 436 3
-16 155 3
-389 712 3
-320 552 4
-263 614 3
-830 696 2
-724 1591 1
-268 227 4
-57 1011 3
-923 307 4
-230 693 2
-102 88 3
-703 471 4
-189 124 5
-804 174 5
-928 288 3
-524 818 3
-346 541 3
-526 127 4
-391 26 5
-934 131 4
-930 756 3
-504 318 5
-54 298 4
-496 483 4
-881 527 3
-500 407 3
-303 597 1
-605 526 5
-746 449 1
-795 168 5
-768 25 4
-489 264 4
-416 1014 3
-637 255 3
-292 525 5
-443 327 4
-379 317 5
-698 516 2
-13 79 3
-363 91 4
-256 5 5
-12 132 5
-465 50 4
-722 476 4
-844 684 3
-224 54 3
-883 703 3
-727 949 3
-373 90 4
-244 1057 4
-584 431 3
-828 753 4
-323 286 3
-521 56 4
-655 1099 3
-332 552 3
-219 215 5
-587 905 3
-549 121 4
-682 209 3
-788 736 3
-344 216 4
-932 185 4
-94 226 2
-136 318 5
-116 259 4
-890 436 3
-54 237 4
-643 179 4
-264 676 3
-478 153 3
-573 134 4
-877 921 4
-308 187 5
-93 276 2
-561 660 3
-639 242 4
-213 100 5
-279 1039 4
-620 795 4
-201 853 4
-815 623 3
-222 228 5
-804 100 5
-887 109 5
-182 111 4
-249 89 5
-728 1355 4
-804 257 5
-823 134 5
-666 169 4
-802 7 5
-75 111 4
-57 1 5
-262 559 3
-66 535 4
-198 1094 1
-251 148 2
-501 222 4
-782 247 1
-201 1069 2
-308 1404 4
-387 697 1
-666 204 3
-215 449 4
-1 116 3
-877 203 4
-320 825 4
-115 48 5
-325 210 2
-788 630 2
-457 173 5
-643 393 4
-307 529 4
-690 649 4
-349 459 4
-106 161 3
-393 22 4
-264 774 2
-1 220 3
-109 451 5
-734 705 4
-393 95 4
-246 257 4
-437 144 2
-561 385 2
-328 595 3
-932 589 5
-833 344 4
-224 1085 1
-757 678 2
-943 569 2
-373 949 4
-661 215 3
-846 127 5
-863 1022 2
-625 204 3
-870 673 5
-479 324 1
-484 79 5
-592 60 4
-666 805 4
-569 473 4
-236 673 4
-446 334 3
-190 310 4
-864 404 4
-663 844 2
-128 380 4
-451 874 4
-49 294 1
-456 188 4
-621 871 3
-865 676 2
-919 295 3
-880 651 5
-627 241 4
-541 15 3
-916 863 3
-271 289 4
-82 338 1
-222 436 4
-659 195 4
-354 185 3
-262 923 4
-902 989 2
-188 629 4
-808 288 3
-445 123 1
-285 455 4
-393 696 4
-788 167 3
-268 273 3
-583 425 5
-263 136 4
-796 1285 4
-401 151 1
-162 544 4
-913 260 1
-58 176 4
-8 183 5
-588 783 4
-870 499 4
-758 303 4
-256 123 2
-22 932 1
-149 874 3
-189 89 5
-850 79 5
-302 309 2
-637 866 3
-234 72 3
-691 650 5
-632 288 3
-648 449 3
-128 924 3
-916 387 4
-727 588 4
-249 12 5
-589 879 4
-416 123 4
-553 519 5
-889 763 4
-846 1518 2
-493 678 3
-846 98 4
-588 735 5
-521 550 3
-350 174 5
-422 551 2
-553 1124 4
-846 789 4
-279 597 5
-926 288 3
-590 298 2
-868 122 3
-887 699 1
-618 1221 2
-130 1245 3
-579 1446 2
-59 508 5
-429 692 3
-77 100 3
-234 1075 3
-301 515 3
-537 923 3
-66 9 4
-311 194 4
-182 123 4
-62 176 5
-758 732 4
-199 687 1
-846 955 3
-786 177 4
-527 474 3
-745 427 4
-940 343 2
-13 790 2
-62 100 4
-290 699 3
-497 24 4
-870 223 4
-474 346 5
-835 50 4
-378 932 2
-286 258 4
-624 262 4
-870 589 4
-561 4 3
-195 831 2
-521 208 3
-890 739 2
-310 24 4
-943 67 4
-263 168 5
-402 182 5
-288 136 5
-320 1052 2
-385 185 5
-904 815 4
-521 827 1
-911 655 5
-868 133 2
-299 529 4
-542 427 5
-231 866 3
-244 171 5
-198 143 3
-690 202 2
-679 748 4
-825 844 2
-883 60 5
-176 286 2
-476 288 4
-74 302 4
-223 717 1
-889 428 4
-873 326 4
-592 357 4
-184 497 4
-669 181 5
-121 156 4
-636 740 4
-405 523 2
-372 674 5
-880 1095 3
-254 419 4
-884 14 4
-159 829 4
-201 943 3
-843 561 4
-13 472 5
-70 15 3
-523 1041 4
-90 863 4
-723 258 4
-416 571 3
-796 1090 4
-589 334 1
-392 269 5
-622 541 2
-733 405 2
-504 428 3
-344 472 3
-130 1017 3
-280 755 2
-365 815 3
-459 257 5
-298 132 5
-301 373 4
-332 660 3
-264 210 5
-151 661 4
-634 109 4
-524 474 4
-102 541 2
-21 990 2
-790 222 3
-257 245 4
-405 606 3
-506 209 4
-178 819 2
-42 1028 4
-835 157 4
-778 239 4
-276 258 5
-880 1157 4
-394 1 4
-361 203 5
-239 8 5
-99 232 4
-265 294 4
-286 209 4
-430 234 4
-95 207 5
-474 238 4
-79 285 5
-807 496 5
-724 272 5
-94 1225 3
-663 284 4
-236 1102 4
-327 418 3
-537 523 3
-592 255 4
-917 50 3
-757 233 3
-387 972 2
-474 275 3
-14 23 5
-49 477 2
-350 136 5
-810 331 4
-782 1012 2
-694 1221 3
-348 628 4
-628 301 4
-709 808 4
-743 288 2
-774 926 1
-339 187 5
-622 69 4
-456 864 4
-870 318 5
-496 53 3
-706 245 3
-249 483 5
-201 644 3
-534 820 3
-483 109 5
-121 14 5
-261 321 3
-435 58 3
-429 118 3
-731 1 2
-551 943 5
-670 945 4
-669 354 1
-790 687 1
-89 1119 3
-833 185 5
-913 95 4
-85 317 3
-459 117 5
-360 1197 3
-707 9 5
-749 1089 3
-909 300 5
-334 265 3
-1 251 4
-403 274 3
-91 527 4
-345 378 4
-264 204 5
-83 751 3
-655 1009 2
-301 443 4
-537 209 4
-472 358 5
-192 1160 4
-588 132 5
-788 200 4
-886 685 2
-286 821 4
-933 154 2
-425 678 1
-755 322 3
-775 343 4
-59 675 5
-488 357 4
-661 144 5
-48 183 5
-21 379 3
-547 302 5
-401 100 4
-345 100 5
-295 227 4
-59 710 3
-933 172 2
-598 895 2
-178 71 4
-704 491 5
-647 77 4
-352 182 5
-407 732 4
-690 554 3
-524 1268 3
-435 587 3
-313 127 5
-539 289 4
-593 79 4
-735 298 4
-897 95 3
-276 67 3
-177 678 3
-890 174 5
-504 106 3
-14 283 4
-763 702 3
-299 255 2
-219 13 1
-692 523 3
-405 1469 1
-315 127 5
-812 286 2
-654 25 1
-656 875 2
-62 357 4
-678 14 3
-749 879 4
-194 265 4
-804 310 4
-724 361 1
-927 385 4
-663 321 5
-727 1231 3
-479 157 5
-31 79 2
-365 108 2
-881 550 3
-455 334 3
-447 559 3
-7 132 5
-916 428 4
-108 255 2
-500 529 4
-231 924 5
-87 568 5
-802 294 4
-267 5 3
-600 583 3
-796 226 3
-880 1059 4
-344 473 4
-743 322 3
-804 63 4
-256 22 5
-388 117 5
-416 105 2
-15 331 3
-882 420 5
-400 306 3
-834 316 5
-267 7 5
-109 949 3
-646 880 3
-311 433 3
-758 24 4
-303 1209 2
-863 268 5
-641 514 4
-712 762 4
-303 939 3
-378 239 3
-773 428 4
-405 603 3
-264 320 4
-305 87 1
-889 282 4
-270 531 4
-634 275 3
-358 469 4
-269 479 4
-295 238 4
-329 100 4
-85 641 4
-650 1039 3
-497 597 3
-429 80 3
-334 337 4
-897 443 5
-594 237 3
-290 1047 4
-276 575 2
-235 463 4
-305 100 3
-655 512 3
-71 514 4
-766 378 4
-459 1038 4
-58 663 2
-580 343 5
-13 111 5
-763 483 4
-214 357 5
-810 342 5
-658 408 5
-269 469 4
-406 382 5
-757 231 2
-867 175 5
-931 312 4
-716 193 5
-7 584 4
-429 136 4
-4 327 5
-35 326 3
-303 1052 2
-577 183 5
-13 630 2
-793 1187 2
-373 265 4
-786 199 4
-468 204 5
-154 479 4
-210 255 4
-644 333 3
-464 176 4
-501 979 3
-358 208 2
-880 232 4
-620 379 4
-887 409 4
-263 250 2
-279 176 3
-911 82 2
-748 186 5
-106 274 3
-693 649 2
-54 634 1
-682 94 3
-653 748 5
-893 240 4
-774 226 2
-674 289 2
-167 1309 1
-60 404 3
-716 1101 5
-128 275 5
-22 430 4
-128 66 3
-738 186 4
-327 269 3
-328 679 2
-318 132 4
-496 1157 1
-603 22 4
-892 238 4
-391 546 3
-790 496 3
-833 208 3
-819 303 4
-711 52 5
-94 684 4
-442 859 3
-768 100 5
-821 274 5
-423 329 3
-650 229 2
-443 358 1
-848 318 5
-399 1178 3
-642 400 4
-510 258 4
-899 176 4
-525 742 3
-788 363 2
-45 7 3
-175 660 3
-465 180 3
-889 1218 4
-610 153 5
-291 202 4
-621 95 4
-75 294 3
-738 470 4
-345 1226 3
-344 663 5
-75 108 4
-494 286 4
-899 471 4
-291 773 3
-749 328 4
-269 608 4
-655 127 5
-450 154 3
-908 591 4
-804 415 3
-435 1217 3
-897 411 5
-512 186 5
-870 697 4
-712 177 2
-655 1266 3
-298 951 4
-303 395 2
-381 418 3
-283 95 5
-83 864 4
-267 685 3
-387 195 4
-304 310 3
-505 724 4
-128 482 4
-201 403 3
-62 763 1
-363 229 3
-730 301 1
-901 1 5
-399 379 3
-795 21 3
-184 134 5
-85 496 4
-276 742 4
-840 526 4
-7 618 4
-328 294 3
-851 475 4
-268 384 3
-155 325 2
-313 484 5
-856 289 1
-296 111 3
-618 597 4
-406 436 4
-275 208 3
-398 117 4
-463 1 1
-922 172 5
-896 479 3
-221 685 3
-504 393 3
-472 109 4
-553 100 5
-586 1273 4
-332 679 5
-158 271 4
-450 1444 4
-77 173 5
-881 103 1
-715 70 3
-330 603 5
-99 676 4
-921 8 3
-621 121 3
-339 81 5
-42 369 4
-616 348 3
-419 617 4
-256 802 3
-407 402 2
-102 286 3
-932 114 5
-406 543 4
-503 1 5
-865 148 3
-284 344 4
-749 622 3
-244 32 2
-703 864 2
-318 968 3
-234 70 3
-748 216 4
-218 208 3
-1 28 4
-352 746 4
-939 1277 5
-708 1061 3
-43 155 4
-807 239 4
-793 844 4
-334 846 3
-148 1039 2
-256 2 5
-567 1204 5
-774 553 2
-838 455 4
-201 896 3
-24 223 5
-224 329 3
-648 25 2
-551 470 5
-145 286 3
-536 318 5
-715 157 4
-924 318 5
-840 143 4
-328 1109 3
-601 154 5
-307 62 3
-254 526 3
-271 178 3
-638 230 5
-332 763 5
-782 1383 3
-100 990 3
-456 427 4
-537 1154 1
-682 50 5
-536 489 4
-848 584 3
-883 228 4
-711 921 5
-592 185 5
-130 379 4
-452 132 2
-436 433 5
-385 488 5
-136 475 4
-586 67 5
-450 650 4
-91

<TRUNCATED>
http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/test/resources/hivemall/mf/ml1k.train.gz
----------------------------------------------------------------------
diff --git a/core/src/test/resources/hivemall/mf/ml1k.train.gz b/core/src/test/resources/hivemall/mf/ml1k.train.gz
new file mode 100644
index 0000000..2f3c384
Binary files /dev/null and b/core/src/test/resources/hivemall/mf/ml1k.train.gz differ



[4/6] incubator-hivemall git commit: Close #13: Implement Kernel Expansion Passive Aggressive Classification

Posted by my...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/test/resources/hivemall/fm/bigdata.tr.txt
----------------------------------------------------------------------
diff --git a/core/src/test/resources/hivemall/fm/bigdata.tr.txt b/core/src/test/resources/hivemall/fm/bigdata.tr.txt
deleted file mode 100644
index 7a00318..0000000
--- a/core/src/test/resources/hivemall/fm/bigdata.tr.txt
+++ /dev/null
@@ -1,200 +0,0 @@
-1 0:0:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:2332:0.3651 6:185:0.3651 7:2569:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:1232:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:1988:0.3651 2:400:0.3651 3:1243:0.3651 4:2183:0.3651 5:2847:0.3651 6:185:0.3651 7:7196:0.3651 8:1343:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:7148:0.50000 16:1828:0.50000
-0 1:7633:0.3651 2:8195:0.3651 3:9952:0.3651 4:9619:0.3651 5:9882:0.3651 6:185:0.3651 7:3479:0.3651 8:3373:0.3651 9:7873:0.3651 10:5989:0.3651 11:1520:0.3651 12:4520:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:3723:0.50000 16:928:0.50000
-1 1:7593:0.3651 2:9126:0.3651 3:9952:0.3651 4:2183:0.3651 5:5525:0.3651 6:5918:0.3651 7:1969:0.3651 8:3240:0.3651 9:3899:0.3651 10:3387:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:8692:0.50000 16:1861:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:3780:0.3651 6:185:0.3651 7:2569:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:2565:0.3651 13:2738:0.3651 14:8813:0.3651 15:6145:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:114:0.3651 6:185:0.3651 7:1424:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:6123:0.3651 13:2738:0.3651 14:2935:0.3651 15:6145:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:319:0.3651 2:5726:0.3651 3:9952:0.3651 4:9619:0.3651 5:9235:0.3651 6:185:0.3651 7:9720:0.3651 8:5805:0.3651 9:3899:0.3651 10:6908:0.3651 11:1520:0.3651 12:3031:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:4342:0.50000 16:3352:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:5947:0.3651 6:185:0.3651 7:1693:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:1232:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-1 1:7633:0.3651 2:8195:0.3651 3:9952:0.3651 4:9619:0.3651 5:9882:0.3651 6:185:0.3651 7:5671:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:4999:0.3651 13:2738:0.3651 14:2935:0.3651 15:1929:0.3651 17:5398:0.28868 17:2434:0.28868 17:3723:0.28868 16:8255:0.28868 16:7755:0.28868 16:928:0.28868
-0 1:2138:0.3651 2:4404:0.3651 3:8672:0.3651 4:2183:0.3651 5:6545:0.3651 6:185:0.3651 7:7547:0.3651 8:4515:0.3651 9:3899:0.3651 10:4465:0.3651 11:1520:0.3651 12:3033:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:6573:0.50000 16:3586:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:642:0.3651 6:185:0.3651 7:1424:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:3252:0.3651 13:2738:0.3651 14:2935:0.3651 15:6145:0.3651 17:9083:0.35355 17:2434:0.35355 16:7755:0.35355 16:872:0.35355
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:2947:0.3651 6:185:0.3651 7:5599:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:8144:0.3651 13:2738:0.3651 14:8813:0.3651 15:1929:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:393:0.3651 2:6291:0.3651 3:9952:0.3651 4:9619:0.3651 5:5029:0.3651 6:185:0.3651 7:9720:0.3651 8:5805:0.3651 9:3899:0.3651 10:6908:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:3824:0.50000 16:1132:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:8369:0.3651 6:185:0.3651 7:5671:0.3651 8:8131:0.3651 9:6160:0.3651 10:215:0.3651 11:1520:0.3651 12:7064:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:8200:0.3651 2:8451:0.3651 3:9474:0.3651 4:2183:0.3651 5:5596:0.3651 6:185:0.3651 7:5599:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:4797:0.3651 13:2549:0.3651 14:2935:0.3651 15:5428:0.3651 17:7026:0.50000 16:8007:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:1624:0.3651 6:185:0.3651 7:5671:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:7759:0.3651 2:7921:0.3651 3:8661:0.3651 4:9619:0.3651 5:6795:0.3651 6:185:0.3651 7:5525:0.3651 8:7752:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:8323:0.50000 16:2974:0.50000
-0 1:7981:0.3651 2:3802:0.3651 3:9952:0.3651 4:2183:0.3651 5:7440:0.3651 6:185:0.3651 7:7785:0.3651 8:8045:0.3651 9:3899:0.3651 10:4465:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2130:0.50000 16:999:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:1734:0.3651 6:185:0.3651 7:8355:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:1232:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:2594:0.3651 6:185:0.3651 7:2569:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:267:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:1707:0.3651 2:50:0.3651 3:7396:0.3651 4:9619:0.3651 5:2738:0.3651 6:5918:0.3651 7:3741:0.3651 8:7752:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:1232:0.3651 13:2738:0.3651 14:2935:0.3651 15:6145:0.3651 17:2288:0.35355 17:287:0.35355 16:7369:0.35355 16:5567:0.35355
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:663:0.3651 6:185:0.3651 7:5671:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:8611:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:1988:0.3651 2:400:0.3651 3:1243:0.3651 4:2183:0.3651 5:1290:0.3651 6:185:0.3651 7:6677:0.3651 8:1343:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:1232:0.3651 13:2738:0.3651 14:8813:0.3651 15:6145:0.3651 17:7148:0.50000 16:1828:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:1820:0.3651 6:185:0.3651 7:8355:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:4055:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:7111:0.3651 6:185:0.3651 7:5671:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:1232:0.3651 13:2738:0.3651 14:8813:0.3651 15:6145:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:1988:0.3651 2:400:0.3651 3:1243:0.3651 4:2183:0.3651 5:3688:0.3651 6:185:0.3651 7:7196:0.3651 8:1343:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:7148:0.50000 16:1828:0.50000
-1 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:3561:0.3651 6:185:0.3651 7:4258:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:5620:0.3651 6:185:0.3651 7:5599:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:9100:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-1 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:1624:0.3651 6:185:0.3651 7:9637:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:416:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:2850:0.3651 2:3395:0.3651 3:9952:0.3651 4:9619:0.3651 5:6394:0.3651 6:185:0.3651 7:6677:0.3651 8:1343:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:8574:0.50000 16:6848:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:7599:0.3651 6:185:0.3651 7:8355:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:6820:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:7879:0.3651 2:8110:0.3651 3:9952:0.3651 4:9619:0.3651 5:6394:0.3651 6:185:0.3651 7:146:0.3651 8:5805:0.3651 9:4487:0.3651 10:6908:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:5225:0.50000 16:675:0.50000
-0 1:7458:0.3651 2:1987:0.3651 3:1243:0.3651 4:9619:0.3651 5:7781:0.3651 6:185:0.3651 7:146:0.3651 8:5805:0.3651 9:3899:0.3651 10:6908:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:4729:0.50000 16:1246:0.50000
-0 1:3712:0.3651 2:896:0.3651 3:9952:0.3651 4:2183:0.3651 5:1501:0.3651 6:185:0.3651 7:9109:0.3651 8:2233:0.3651 9:2032:0.3651 10:4465:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:4380:0.50000 16:1867:0.50000
-0 1:8862:0.3651 2:3807:0.3651 3:3785:0.3651 4:2183:0.3651 5:6963:0.3651 6:185:0.3651 7:3741:0.3651 8:7752:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:7732:0.50000 16:986:0.50000
-0 1:3136:0.3651 2:7449:0.3651 3:1243:0.3651 4:9619:0.3651 5:2738:0.3651 6:5918:0.3651 7:3741:0.3651 8:7752:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:8736:0.50000 16:1977:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:2863:0.3651 6:185:0.3651 7:4258:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:9971:0.3651 13:2738:0.3651 14:2935:0.3651 15:9292:0.3651 17:5398:0.11396 17:9083:0.02849 17:9916:0.02849 17:8047:0.02849 17:2434:0.48433 16:8255:0.11396 16:7755:0.48433 16:9406:0.02849 16:8226:0.02849 16:1826:0.02849
-0 1:2850:0.3651 2:3395:0.3651 3:9952:0.3651 4:9619:0.3651 5:1501:0.3651 6:185:0.3651 7:8893:0.3651 8:4421:0.3651 9:3899:0.3651 10:7582:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:8574:0.50000 16:6848:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:5390:0.3651 6:185:0.3651 7:2569:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:4961:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:6795:0.3651 6:185:0.3651 7:1424:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:1332:0.3651 2:4059:0.3651 3:9952:0.3651 4:2183:0.3651 5:1014:0.3651 6:185:0.3651 7:3741:0.3651 8:7752:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:9338:0.50000 16:100:0.50000
-1 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:1947:0.3651 6:185:0.3651 7:5671:0.3651 8:8131:0.3651 9:6160:0.3651 10:215:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:9882:0.3651 6:185:0.3651 7:1424:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:1232:0.3651 13:2738:0.3651 14:2935:0.3651 15:6145:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:2146:0.3651 2:4357:0.3651 3:9952:0.3651 4:9619:0.3651 5:9416:0.3651 6:185:0.3651 7:3076:0.3651 8:260:0.3651 9:4522:0.3651 10:953:0.3651 11:1520:0.3651 12:1232:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:663:0.50000 16:2035:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:2646:0.3651 6:185:0.3651 7:8355:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:7685:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:7501:0.3651 2:439:0.3651 3:1243:0.3651 4:2183:0.3651 5:2654:0.3651 6:185:0.3651 7:1398:0.3651 8:4331:0.3651 9:5483:0.3651 10:5620:0.3651 11:1520:0.3651 12:2052:0.3651 13:2738:0.3651 14:2935:0.3651 15:6858:0.3651 17:9083:0.40825 17:9036:0.20412 17:2434:0.20412 16:7755:0.25000 16:9111:0.25000 16:7097:0.25000 16:872:0.25000
-1 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:1734:0.3651 6:185:0.3651 7:4258:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:3641:0.3651 2:5431:0.3651 3:9952:0.3651 4:2183:0.3651 5:3688:0.3651 6:185:0.3651 7:7196:0.3651 8:1343:0.3651 9:5531:0.3651 10:9493:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:7388:0.50000 16:8185:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:2594:0.3651 6:185:0.3651 7:8355:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:1175:0.3651 13:2738:0.3651 14:2935:0.3651 15:1468:0.3651 17:9083:0.03835 17:2434:0.49853 16:7755:0.49853 16:8226:0.03835
-0 1:7501:0.3651 2:439:0.3651 3:1243:0.3651 4:2183:0.3651 5:1624:0.3651 6:185:0.3651 7:1398:0.3651 8:4331:0.3651 9:6160:0.3651 10:5620:0.3651 11:1520:0.3651 12:4520:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:9083:0.50000 16:872:0.50000
-0 1:399:0.3651 2:2157:0.3651 3:9952:0.3651 4:9619:0.3651 5:9745:0.3651 6:185:0.3651 7:5525:0.3651 8:7752:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:513:0.50000 16:1334:0.50000
-1 1:206:0.3651 2:674:0.3651 3:9952:0.3651 4:9619:0.3651 5:9591:0.3651 6:185:0.3651 7:1693:0.3651 8:8131:0.3651 9:6160:0.3651 10:215:0.3651 11:1520:0.3651 12:1089:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:5398:0.50000 16:8255:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:7805:0.3651 6:185:0.3651 7:5599:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:6298:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-1 1:1988:0.3651 2:400:0.3651 3:1243:0.3651 4:2183:0.3651 5:8118:0.3651 6:185:0.3651 7:6677:0.3651 8:1343:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:7148:0.50000 16:1828:0.50000
-0 1:1988:0.3651 2:400:0.3651 3:1243:0.3651 4:2183:0.3651 5:2863:0.3651 6:185:0.3651 7:7196:0.3651 8:1343:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:1232:0.3651 13:2738:0.3651 14:8813:0.3651 15:6145:0.3651 17:7148:0.50000 16:1828:0.50000
-0 1:6381:0.3651 2:3648:0.3651 3:9952:0.3651 4:2183:0.3651 5:4923:0.3651 6:185:0.3651 7:5525:0.3651 8:7752:0.3651 9:2751:0.3651 10:9493:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:6323:0.50000 16:6611:0.50000
-0 1:4506:0.3651 2:441:0.3651 3:660:0.3651 4:2183:0.3651 5:9882:0.3651 6:185:0.3651 7:3741:0.3651 8:7752:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:1232:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:6180:0.50000 16:8742:0.50000
-0 1:1988:0.3651 2:400:0.3651 3:1243:0.3651 4:2183:0.3651 5:8118:0.3651 6:185:0.3651 7:867:0.3651 8:1343:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:7148:0.50000 16:1828:0.50000
-0 1:1988:0.3651 2:400:0.3651 3:1243:0.3651 4:2183:0.3651 5:2863:0.3651 6:185:0.3651 7:7196:0.3651 8:1343:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:7685:0.3651 13:2738:0.3651 14:8813:0.3651 15:6858:0.3651 17:7148:0.50000 16:1828:0.50000
-1 1:7633:0.3651 2:8195:0.3651 3:9952:0.3651 4:9619:0.3651 5:6795:0.3651 6:185:0.3651 7:6677:0.3651 8:1343:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:3723:0.50000 16:928:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:6832:0.3651 6:185:0.3651 7:1424:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:1392:0.3651 6:5918:0.3651 7:2166:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:512:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:1392:0.3651 6:5918:0.3651 7:2569:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:9616:0.3651 2:4389:0.3651 3:9952:0.3651 4:2183:0.3651 5:9882:0.3651 6:185:0.3651 7:5525:0.3651 8:7752:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:457:0.50000 16:3204:0.50000
-0 1:9616:0.3651 2:4389:0.3651 3:9952:0.3651 4:2183:0.3651 5:5306:0.3651 6:185:0.3651 7:3741:0.3651 8:7752:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:457:0.50000 16:3204:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:421:0.3651 6:185:0.3651 7:8355:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:143:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-1 1:7501:0.3651 2:439:0.3651 3:1243:0.3651 4:2183:0.3651 5:3077:0.3651 6:185:0.3651 7:1398:0.3651 8:4331:0.3651 9:3899:0.3651 10:5620:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:9083:0.50000 16:872:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:7111:0.3651 6:185:0.3651 7:2166:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:1232:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:9882:0.3651 6:185:0.3651 7:2166:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:6746:0.3651 13:2738:0.3651 14:8813:0.3651 15:8864:0.3651 17:5398:0.04730 17:2288:0.00788 17:9916:0.02365 17:8047:0.01577 17:2434:0.49669 17:7388:0.00788 17:3824:0.00788 17:2641:0.00788 16:6996:0.00788 16:7369:0.00788 16:1132:0.00788 16:8255:0.04730 16:7755:0.49669 16:9406:0.02365 16:1826:0.01577 16:8846:0.00788
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:7111:0.3651 6:185:0.3651 7:8355:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:4490:0.3651 13:2738:0.3651 14:2935:0.3651 15:6145:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:1988:0.3651 2:400:0.3651 3:1243:0.3651 4:2183:0.3651 5:888:0.3651 6:185:0.3651 7:867:0.3651 8:1343:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:1232:0.3651 13:2738:0.3651 14:8813:0.3651 15:6145:0.3651 17:7148:0.50000 16:1828:0.50000
-0 1:6669:0.3651 2:6533:0.3651 3:7086:0.3651 4:2183:0.3651 5:6629:0.3651 6:185:0.3651 7:8355:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:894:0.50000 16:9716:0.50000
-0 1:1988:0.3651 2:400:0.3651 3:1243:0.3651 4:2183:0.3651 5:6963:0.3651 6:185:0.3651 7:7196:0.3651 8:1343:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:7148:0.50000 16:1828:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:9882:0.3651 6:185:0.3651 7:5671:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:1742:0.3651 13:2738:0.3651 14:2935:0.3651 15:8973:0.3651 17:5800:0.09449 17:9792:0.09449 17:2434:0.47246 17:3723:0.09449 16:7755:0.47246 16:9568:0.09449 16:928:0.09449 16:7400:0.09449
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:7111:0.3651 6:185:0.3651 7:5671:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:9882:0.3651 6:185:0.3651 7:1693:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:6123:0.3651 13:2738:0.3651 14:2935:0.3651 15:3607:0.3651 17:5398:0.07836 17:8549:0.00784 17:8047:0.00784 17:2434:0.49370 16:8255:0.07836 16:7755:0.49370 16:9926:0.00784 16:1826:0.00784
-0 1:2850:0.3651 2:3395:0.3651 3:9952:0.3651 4:9619:0.3651 5:9311:0.3651 6:185:0.3651 7:867:0.3651 8:1343:0.3651 9:4487:0.3651 10:9493:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:8574:0.50000 16:6848:0.50000
-0 1:2426:0.3651 2:5064:0.3651 3:9474:0.3651 4:2183:0.3651 5:5011:0.3651 6:185:0.3651 7:5533:0.3651 8:5531:0.3651 9:3899:0.3651 10:3330:0.3651 11:1520:0.3651 12:9714:0.3651 13:2549:0.3651 14:2935:0.3651 15:5428:0.3651 17:390:0.50000 16:7250:0.50000
-0 1:2548:0.3651 2:2379:0.3651 3:8672:0.3651 4:9619:0.3651 5:1445:0.3651 6:185:0.3651 7:7818:0.3651 8:4231:0.3651 9:638:0.3651 10:953:0.3651 11:1520:0.3651 12:4838:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2641:0.50000 16:1267:0.50000
-1 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:2594:0.3651 6:185:0.3651 7:9637:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:6724:0.3651 6:5918:0.3651 7:2569:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:143:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-1 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:1624:0.3651 6:185:0.3651 7:8355:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-1 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:9882:0.3651 6:185:0.3651 7:5599:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:7685:0.3651 13:2738:0.3651 14:8813:0.3651 15:6145:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:1988:0.3651 2:400:0.3651 3:1243:0.3651 4:2183:0.3651 5:888:0.3651 6:185:0.3651 7:867:0.3651 8:1343:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:1232:0.3651 13:2738:0.3651 14:8813:0.3651 15:6145:0.3651 17:7148:0.50000 16:1828:0.50000
-1 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:9882:0.3651 6:185:0.3651 7:1693:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:4520:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-1 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:2594:0.3651 6:185:0.3651 7:5671:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:9638:0.3651 13:2738:0.3651 14:2935:0.3651 15:192:0.3651 17:5398:0.02491 17:9083:0.02491 17:8047:0.02491 17:2434:0.49814 16:8255:0.02491 16:7755:0.49814 16:1826:0.02491 16:872:0.02491
-1 1:319:0.3651 2:5726:0.3651 3:9952:0.3651 4:9619:0.3651 5:3780:0.3651 6:185:0.3651 7:8018:0.3651 8:2233:0.3651 9:4861:0.3651 10:4465:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:4342:0.50000 16:3352:0.50000
-0 1:464:0.3651 2:5202:0.3651 3:7396:0.3651 4:9619:0.3651 5:2738:0.3651 6:5918:0.3651 7:5525:0.3651 8:7752:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:9100:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2288:0.50000 16:7369:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:5661:0.3651 6:5918:0.3651 7:9637:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:416:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-1 1:1409:0.3651 2:664:0.3651 3:8672:0.3651 4:2183:0.3651 5:5909:0.3651 6:185:0.3651 7:5533:0.3651 8:5531:0.3651 9:3899:0.3651 10:3330:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:911:0.50000 16:9686:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:7111:0.3651 6:185:0.3651 7:8355:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:5811:0.3651 13:2738:0.3651 14:2935:0.3651 15:5344:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:9214:0.3651 2:2037:0.3651 3:8661:0.3651 4:9619:0.3651 5:649:0.3651 6:185:0.3651 7:5525:0.3651 8:7752:0.3651 9:3151:0.3651 10:9493:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:8036:0.50000 16:3778:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:6629:0.3651 6:185:0.3651 7:5599:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:8072:0.3651 13:2738:0.3651 14:2935:0.3651 15:6145:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:3652:0.3651 6:5918:0.3651 7:9637:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:1988:0.3651 2:400:0.3651 3:1243:0.3651 4:2183:0.3651 5:6963:0.3651 6:185:0.3651 7:7196:0.3651 8:1343:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:7148:0.50000 16:1828:0.50000
-1 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:9774:0.3651 6:185:0.3651 7:4258:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:1409:0.3651 6:185:0.3651 7:5671:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:4786:0.3651 13:2738:0.3651 14:2935:0.3651 15:5344:0.3651 17:9163:0.12127 17:2434:0.48507 16:5318:0.12127 16:7755:0.48507
-1 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:9493:0.3651 6:185:0.3651 7:5599:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:1232:0.3651 13:2738:0.3651 14:8813:0.3651 15:6145:0.3651 17:2434:0.50000 16:7755:0.50000
-1 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:1438:0.3651 6:185:0.3651 7:8355:0.3651 8:8131:0.3651 9:6160:0.3651 10:215:0.3651 11:1520:0.3651 12:4999:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:7633:0.3651 2:8195:0.3651 3:9952:0.3651 4:9619:0.3651 5:5947:0.3651 6:185:0.3651 7:8937:0.3651 8:7183:0.3651 9:3899:0.3651 10:5620:0.3651 11:1520:0.3651 12:1232:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:3723:0.50000 16:928:0.50000
-1 1:7633:0.3651 2:8195:0.3651 3:9952:0.3651 4:9619:0.3651 5:9200:0.3651 6:185:0.3651 7:142:0.3651 8:1343:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:3723:0.50000 16:928:0.50000
-0 1:7682:0.3651 2:8451:0.3651 3:9474:0.3651 4:2183:0.3651 5:753:0.3651 6:185:0.3651 7:2334:0.3651 8:5010:0.3651 9:3899:0.3651 10:3387:0.3651 11:1520:0.3651 12:9714:0.3651 13:2549:0.3651 14:2935:0.3651 15:5428:0.3651 17:7026:0.50000 16:5908:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:1729:0.3651 6:185:0.3651 7:2569:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:1232:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-1 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:6724:0.3651 6:5918:0.3651 7:1424:0.3651 8:8131:0.3651 9:6160:0.3651 10:215:0.3651 11:1520:0.3651 12:4999:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:351:0.3651 2:2613:0.3651 3:9952:0.3651 4:2183:0.3651 5:1820:0.3651 6:185:0.3651 7:6677:0.3651 8:1343:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:5547:0.3651 13:2738:0.3651 14:2935:0.3651 15:1929:0.3651 17:3723:0.44721 17:4184:0.22361 16:928:0.44721 16:3348:0.22361
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:6745:0.3651 6:185:0.3651 7:8355:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-1 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:9774:0.3651 6:185:0.3651 7:5599:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:9100:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-1 1:5116:0.3651 2:6465:0.3651 3:9952:0.3651 4:2183:0.3651 5:9882:0.3651 6:185:0.3651 7:5525:0.3651 8:7752:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:8795:0.50000 16:9899:0.50000
-1 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:7111:0.3651 6:185:0.3651 7:5599:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:8222:0.3651 6:185:0.3651 7:9637:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:5642:0.3651 13:2738:0.3651 14:2935:0.3651 15:6885:0.3651 17:5398:0.20604 17:9083:0.06868 17:3121:0.03434 17:2434:0.44642 17:894:0.03434 17:3723:0.03434 16:9716:0.03434 16:872:0.06868 16:928:0.03434 16:8255:0.20604 16:7755:0.44642 16:6900:0.03434
-0 1:246:0.3651 2:3451:0.3651 3:6764:0.3651 4:9619:0.3651 5:5463:0.3651 6:185:0.3651 7:3741:0.3651 8:7752:0.3651 9:9603:0.3651 10:9493:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:4694:0.50000 16:6777:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:9653:0.3651 6:185:0.3651 7:9637:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:5471:0.3651 13:2738:0.3651 14:2935:0.3651 15:6145:0.3651 17:2434:0.50000 16:7755:0.50000
-1 1:1988:0.3651 2:400:0.3651 3:1243:0.3651 4:2183:0.3651 5:573:0.3651 6:185:0.3651 7:7196:0.3651 8:1343:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:7685:0.3651 13:2738:0.3651 14:9125:0.3651 15:6858:0.3651 17:7148:0.50000 16:1828:0.50000
-0 1:206:0.3651 2:674:0.3651 3:9952:0.3651 4:2183:0.3651 5:5613:0.3651 6:185:0.3651 7:5525:0.3651 8:7752:0.3651 9:9554:0.3651 10:9493:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:5398:0.50000 16:8255:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:8222:0.3651 6:185:0.3651 7:1693:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:4961:0.3651 13:2738:0.3651 14:2935:0.3651 15:8453:0.3651 17:5398:0.20788 17:2288:0.01890 17:7824:0.01890 17:2434:0.45356 17:9083:0.01890 16:7369:0.01917 16:872:0.01917 16:4991:0.01917 16:8255:0.19174 16:7755:0.46018 16:1236:0.01917
-0 1:6462:0.3651 2:2379:0.3651 3:8672:0.3651 4:9619:0.3651 5:2401:0.3651 6:185:0.3651 7:9637:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:6746:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2641:0.50000 16:58:0.50000
-1 1:8379:0.3651 2:1847:0.3651 3:9952:0.3651 4:2183:0.3651 5:5947:0.3651 6:185:0.3651 7:9635:0.3651 8:1241:0.3651 9:7873:0.3651 10:4465:0.3651 11:1520:0.3651 12:4490:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:8047:0.50000 16:1826:0.50000
-0 1:1988:0.3651 2:400:0.3651 3:1243:0.3651 4:2183:0.3651 5:2310:0.3651 6:185:0.3651 7:867:0.3651 8:1343:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:9100:0.3651 13:2738:0.3651 14:385:0.3651 15:1929:0.3651 17:7148:0.50000 16:1828:0.50000
-0 1:8850:0.3651 2:7834:0.3651 3:9952:0.3651 4:2183:0.3651 5:1392:0.3651 6:5918:0.3651 7:5525:0.3651 8:7752:0.3651 9:381:0.3651 10:9493:0.3651 11:1520:0.3651 12:1232:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:650:0.50000 16:7547:0.50000
-1 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:3688:0.3651 6:185:0.3651 7:5599:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:9882:0.3651 6:185:0.3651 7:4258:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-1 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:1176:0.3651 6:185:0.3651 7:1693:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:8982:0.3651 13:2738:0.3651 14:2935:0.3651 15:8973:0.3651 17:5398:0.18257 17:3121:0.09129 17:2434:0.45644 16:8255:0.18257 16:7755:0.45644 16:6900:0.09129
-1 1:206:0.3651 2:674:0.3651 3:9952:0.3651 4:9619:0.3651 5:488:0.3651 6:185:0.3651 7:7836:0.3651 8:653:0.3651 9:6160:0.3651 10:5620:0.3651 11:1520:0.3651 12:6229:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:5398:0.50000 16:8255:0.50000
-0 1:9945:0.3651 2:3412:0.3651 3:6764:0.3651 4:9619:0.3651 5:5480:0.3651 6:185:0.3651 7:5525:0.3651 8:7752:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:474:0.50000 16:8910:0.50000
-0 1:3230:0.3651 2:2905:0.3651 3:9952:0.3651 4:9619:0.3651 5:1501:0.3651 6:185:0.3651 7:9109:0.3651 8:2233:0.3651 9:4487:0.3651 10:4465:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:8976:0.50000 16:9067:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:1392:0.3651 6:5918:0.3651 7:2166:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:582:0.3651 13:2738:0.3651 14:2935:0.3651 15:6145:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:9882:0.3651 6:185:0.3651 7:4258:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-1 1:1018:0.3651 2:7079:0.3651 3:1243:0.3651 4:9619:0.3651 5:9882:0.3651 6:185:0.3651 7:5525:0.3651 8:7752:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:3457:0.50000 16:5372:0.50000
-0 1:513:0.3651 2:8451:0.3651 3:9474:0.3651 4:2183:0.3651 5:4154:0.3651 6:185:0.3651 7:5932:0.3651 8:1786:0.3651 9:1861:0.3651 10:4062:0.3651 11:1520:0.3651 12:9714:0.3651 13:2549:0.3651 14:2935:0.3651 15:5428:0.3651 17:7026:0.50000 16:1418:0.50000
-0 1:3086:0.3651 2:894:0.3651 3:9931:0.3651 4:2183:0.3651 5:7912:0.3651 6:185:0.3651 7:9051:0.3651 8:4637:0.3651 9:6390:0.3651 10:2106:0.3651 11:1520:0.3651 12:4999:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:8914:0.50000 16:650:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:1857:0.3651 6:5918:0.3651 7:2569:0.3651 8:8131:0.3651 9:6160:0.3651 10:215:0.3651 11:1520:0.3651 12:4520:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:9882:0.3651 6:185:0.3651 7:1424:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-1 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:6052:0.3651 6:185:0.3651 7:8355:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:9100:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-1 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:2863:0.3651 6:185:0.3651 7:1693:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-1 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:7111:0.3651 6:185:0.3651 7:9637:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:1232:0.3651 13:2738:0.3651 14:8813:0.3651 15:6145:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:4502:0.3651 2:2379:0.3651 3:9952:0.3651 4:9619:0.3651 5:6795:0.3651 6:185:0.3651 7:2569:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2641:0.50000 16:8846:0.50000
-0 1:9214:0.3651 2:2037:0.3651 3:8661:0.3651 4:9619:0.3651 5:2654:0.3651 6:185:0.3651 7:3741:0.3651 8:7752:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:8036:0.50000 16:3778:0.50000
-0 1:6825:0.3651 2:1623:0.3651 3:9952:0.3651 4:2183:0.3651 5:3169:0.3651 6:185:0.3651 7:7196:0.3651 8:1343:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:9100:0.3651 13:2738:0.3651 14:8813:0.3651 15:6145:0.3651 17:6994:0.50000 16:4008:0.50000
-0 1:7633:0.3651 2:8195:0.3651 3:9952:0.3651 4:9619:0.3651 5:5318:0.3651 6:185:0.3651 7:3479:0.3651 8:3373:0.3651 9:3899:0.3651 10:5989:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:3723:0.50000 16:928:0.50000
-0 1:206:0.3651 2:674:0.3651 3:9952:0.3651 4:9619:0.3651 5:3780:0.3651 6:185:0.3651 7:7836:0.3651 8:653:0.3651 9:3899:0.3651 10:5620:0.3651 11:1520:0.3651 12:766:0.3651 13:2738:0.3651 14:2935:0.3651 15:6325:0.3651 17:5398:0.13245 17:8047:0.13245 17:2434:0.46359 16:8255:0.13245 16:7755:0.46359 16:1826:0.13245
-0 1:319:0.3651 2:5726:0.3651 3:9952:0.3651 4:9619:0.3651 5:613:0.3651 6:185:0.3651 7:146:0.3651 8:5805:0.3651 9:5189:0.3651 10:6908:0.3651 11:1520:0.3651 12:1250:0.3651 13:2738:0.3651 14:2935:0.3651 15:6145:0.3651 17:4342:0.50000 16:3352:0.50000
-0 1:206:0.3651 2:674:0.3651 3:9952:0.3651 4:9619:0.3651 5:8142:0.3651 6:185:0.3651 7:7836:0.3651 8:653:0.3651 9:3899:0.3651 10:5620:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:5398:0.50000 16:8255:0.50000
-1 1:206:0.3651 2:674:0.3651 3:9952:0.3651 4:9619:0.3651 5:6629:0.3651 6:185:0.3651 7:7836:0.3651 8:653:0.3651 9:5483:0.3651 10:5620:0.3651 11:1520:0.3651 12:7685:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:5398:0.50000 16:8255:0.50000
-0 1:4993:0.3651 2:8384:0.3651 3:9952:0.3651 4:9619:0.3651 5:4729:0.3651 6:5918:0.3651 7:3741:0.3651 8:7752:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:9100:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:8700:0.50000 16:2866:0.50000
-0 1:7759:0.3651 2:7921:0.3651 3:8661:0.3651 4:9619:0.3651 5:2654:0.3651 6:185:0.3651 7:3741:0.3651 8:7752:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:8323:0.50000 16:2974:0.50000
-0 1:2850:0.3651 2:3395:0.3651 3:9952:0.3651 4:9619:0.3651 5:3169:0.3651 6:185:0.3651 7:3735:0.3651 8:2685:0.3651 9:3483:0.3651 10:7582:0.3651 11:1520:0.3651 12:9100:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:8574:0.50000 16:6848:0.50000
-0 1:6462:0.3651 2:2379:0.3651 3:8672:0.3651 4:9619:0.3651 5:7305:0.3651 6:185:0.3651 7:9637:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:1232:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2641:0.50000 16:58:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:9882:0.3651 6:185:0.3651 7:5671:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:4786:0.3651 13:2738:0.3651 14:2935:0.3651 15:5872:0.3651 17:5398:0.08817 17:5800:0.00980 17:8047:0.03919 17:2434:0.48982 17:8700:0.00980 17:3824:0.00980 17:8844:0.01959 17:7571:0.00980 16:338:0.01959 16:7433:0.00980 16:2866:0.00980 16:1132:0.00980 16:8255:0.08817 16:7755:0.48982 16:1826:0.03919 16:9568:0.00980
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:6629:0.3651 6:185:0.3651 7:5599:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:8982:0.3651 13:2738:0.3651 14:2935:0.3651 15:5007:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:1332:0.3651 2:4059:0.3651 3:9952:0.3651 4:2183:0.3651 5:9882:0.3651 6:185:0.3651 7:3741:0.3651 8:7752:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:9100:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:9338:0.50000 16:100:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:7680:0.3651 6:185:0.3651 7:2569:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:393:0.3651 2:6291:0.3651 3:9952:0.3651 4:9619:0.3651 5:9653:0.3651 6:185:0.3651 7:6690:0.3651 8:6571:0.3651 9:3899:0.3651 10:953:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:3824:0.50000 16:1132:0.50000
-0 1:1228:0.3651 2:2179:0.3651 3:9474:0.3651 4:2183:0.3651 5:1036:0.3651 6:185:0.3651 7:1969:0.3651 8:3240:0.3651 9:4487:0.3651 10:3387:0.3651 11:1520:0.3651 12:9714:0.3651 13:2549:0.3651 14:2935:0.3651 15:5428:0.3651 17:4277:0.50000 16:2625:0.50000
-1 1:1409:0.3651 2:664:0.3651 3:8672:0.3651 4:2183:0.3651 5:6795:0.3651 6:185:0.3651 7:5533:0.3651 8:5531:0.3651 9:3899:0.3651 10:3330:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:911:0.50000 16:9686:0.50000
-0 1:7952:0.3651 2:2392:0.3651 3:9952:0.3651 4:9619:0.3651 5:5029:0.3651 6:185:0.3651 7:9720:0.3651 8:5805:0.3651 9:3483:0.3651 10:6908:0.3651 11:1520:0.3651 12:7685:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:7571:0.50000 16:7433:0.50000
-1 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:9882:0.3651 6:185:0.3651 7:4258:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:8222:0.3651 6:185:0.3651 7:8355:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:9100:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:7458:0.3651 2:1987:0.3651 3:1243:0.3651 4:9619:0.3651 5:6873:0.3651 6:185:0.3651 7:9720:0.3651 8:5805:0.3651 9:5189:0.3651 10:6908:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:4729:0.50000 16:1246:0.50000
-1 1:6097:0.3651 2:2063:0.3651 3:9952:0.3651 4:9619:0.3651 5:4039:0.3651 6:185:0.3651 7:146:0.3651 8:5805:0.3651 9:2032:0.3651 10:6908:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2100:0.50000 16:357:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:610:0.3651 6:5918:0.3651 7:2166:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-1 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:8786:0.3651 6:185:0.3651 7:4258:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-1 1:449:0.3651 2:439:0.3651 3:1243:0.3651 4:2183:0.3651 5:9200:0.3651 6:185:0.3651 7:1398:0.3651 8:4331:0.3651 9:5483:0.3651 10:5620:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:9083:0.50000 16:8226:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:9882:0.3651 6:185:0.3651 7:4258:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:1988:0.3651 2:400:0.3651 3:1243:0.3651 4:2183:0.3651 5:3777:0.3651 6:185:0.3651 7:6677:0.3651 8:1343:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:1232:0.3651 13:2738:0.3651 14:8813:0.3651 15:6145:0.3651 17:7148:0.50000 16:1828:0.50000
-1 1:2146:0.3651 2:4357:0.3651 3:9952:0.3651 4:9619:0.3651 5:21:0.3651 6:185:0.3651 7:7714:0.3651 8:7470:0.3651 9:4698:0.3651 10:8245:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:663:0.50000 16:2035:0.50000
-0 1:48:0.3651 2:8451:0.3651 3:9474:0.3651 4:2183:0.3651 5:8216:0.3651 6:185:0.3651 7:1969:0.3651 8:3240:0.3651 9:4861:0.3651 10:3387:0.3651 11:1520:0.3651 12:9714:0.3651 13:2549:0.3651 14:2935:0.3651 15:5428:0.3651 17:7026:0.50000 16:8965:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:9774:0.3651 6:185:0.3651 7:4258:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:5980:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:9653:0.3651 6:185:0.3651 7:2166:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:405:0.3651 13:2738:0.3651 14:2935:0.3651 15:6145:0.3651 17:2434:0.50000 16:7755:0.50000
-1 1:206:0.3651 2:674:0.3651 3:9952:0.3651 4:9619:0.3651 5:6963:0.3651 6:185:0.3651 7:7230:0.3651 8:653:0.3651 9:3899:0.3651 10:5620:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:5398:0.50000 16:8255:0.50000
-0 1:513:0.3651 2:8451:0.3651 3:9474:0.3651 4:2183:0.3651 5:4154:0.3651 6:185:0.3651 7:4692:0.3651 8:1786:0.3651 9:1861:0.3651 10:4062:0.3651 11:1520:0.3651 12:9714:0.3651 13:2549:0.3651 14:2935:0.3651 15:5428:0.3651 17:7026:0.50000 16:1418:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:2863:0.3651 6:185:0.3651 7:8355:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:563:0.3651 13:2738:0.3651 14:2935:0.3651 15:2757:0.3651 17:9083:0.12127 17:2434:0.48507 16:7755:0.48507 16:9111:0.12127
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:9882:0.3651 6:185:0.3651 7:1693:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:5250:0.3651 2:6423:0.3651 3:6764:0.3651 4:9619:0.3651 5:8408:0.3651 6:185:0.3651 7:5525:0.3651 8:7752:0.3651 9:381:0.3651 10:9493:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:5316:0.50000 16:5065:0.50000
-1 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:4658:0.3651 6:185:0.3651 7:5599:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:3561:0.3651 6:185:0.3651 7:5671:0.3651 8:8131:0.3651 9:6160:0.3651 10:215:0.3651 11:1520:0.3651 12:267:0.3651 13:2738:0.3651 14:8813:0.3651 15:6145:0.3651 17:2434:0.50000 16:7755:0.50000
-1 1:7501:0.3651 2:439:0.3651 3:1243:0.3651 4:2183:0.3651 5:1624:0.3651 6:185:0.3651 7:1398:0.3651 8:4331:0.3651 9:5483:0.3651 10:5620:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:9083:0.50000 16:872:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:421:0.3651 6:185:0.3651 7:5599:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:143:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:6795:0.3651 6:185:0.3651 7:4258:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:143:0.3651 13:2738:0.3651 14:2935:0.3651 15:1929:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:1382:0.3651 2:9270:0.3651 3:9952:0.3651 4:2183:0.3651 5:3169:0.3651 6:185:0.3651 7:7196:0.3651 8:1343:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:4999:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:8737:0.50000 16:232:0.50000
-0 1:6076:0.3651 2:1921:0.3651 3:8672:0.3651 4:2183:0.3651 5:1750:0.3651 6:185:0.3651 7:6704:0.3651 8:8580:0.3651 9:3899:0.3651 10:6805:0.3651 11:1520:0.3651 12:1089:0.3651 13:2738:0.3651 14:2935:0.3651 15:1929:0.3651 17:2004:0.28868 17:8616:0.28868 17:9990:0.28868 16:2198:0.28868 16:1762:0.28868 16:2803:0.28868
-0 1:1988:0.3651 2:400:0.3651 3:1243:0.3651 4:2183:0.3651 5:573:0.3651 6:185:0.3651 7:7196:0.3651 8:1343:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:7685:0.3651 13:2738:0.3651 14:9125:0.3651 15:6858:0.3651 17:7148:0.50000 16:1828:0.50000
-1 1:6381:0.3651 2:3648:0.3651 3:9952:0.3651 4:2183:0.3651 5:1501:0.3651 6:185:0.3651 7:3741:0.3651 8:7752:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:6323:0.50000 16:6611:0.50000
-0 1:1988:0.3651 2:400:0.3651 3:1243:0.3651 4:2183:0.3651 5:2310:0.3651 6:185:0.3651 7:867:0.3651 8:1343:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:7148:0.50000 16:1828:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:7305:0.3651 6:185:0.3651 7:2166:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:2850:0.3651 2:3395:0.3651 3:9952:0.3651 4:9619:0.3651 5:2694:0.3651 6:185:0.3651 7:3735:0.3651 8:2685:0.3651 9:3899:0.3651 10:7582:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:8574:0.50000 16:6848:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:2431:0.3651 6:185:0.3651 7:2166:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:9100:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:7633:0.3651 2:8195:0.3651 3:9952:0.3651 4:9619:0.3651 5:5383:0.3651 6:185:0.3651 7:932:0.3651 8:5166:0.3651 9:8193:0.3651 10:4465:0.3651 11:1520:0.3651 12:4999:0.3651 13:2738:0.3651 14:2935:0.3651 15:6145:0.3651 17:3723:0.50000 16:928:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:9882:0.3651 6:185:0.3651 7:4258:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:1232:0.3651 13:2738:0.3651 14:8813:0.3651 15:6145:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:5116:0.3651 2:6465:0.3651 3:9952:0.3651 4:2183:0.3651 5:6506:0.3651 6:185:0.3651 7:3741:0.3651 8:7752:0.3651 9:3899:0.3651 10:9493:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:8795:0.50000 16:9899:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:3122:0.3651 6:185:0.3651 7:5671:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:9100:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:9882:0.3651 6:185:0.3651 7:2569:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:7685:0.3651 13:2738:0.3651 14:2935:0.3651 15:6145:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:7111:0.3651 6:185:0.3651 7:1424:0.3651 8:8131:0.3651 9:5483:0.3651 10:215:0.3651 11:1520:0.3651 12:1232:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:249:0.3651 6:185:0.3651 7:2569:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:6187:0.3651 13:2738:0.3651 14:2935:0.3651 15:4252:0.3651 17:2867:0.03835 17:2434:0.49853 16:1725:0.03835 16:7755:0.49853
-0 1:2146:0.3651 2:4357:0.3651 3:9952:0.3651 4:9619:0.3651 5:2654:0.3651 6:185:0.3651 7:3076:0.3651 8:260:0.3651 9:6391:0.3651 10:953:0.3651 11:1520:0.3651 12:143:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:663:0.50000 16:2035:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:3561:0.3651 6:185:0.3651 7:9637:0.3651 8:8131:0.3651 9:6160:0.3651 10:215:0.3651 11:1520:0.3651 12:267:0.3651 13:2738:0.3651 14:8813:0.3651 15:6145:0.3651 17:2434:0.50000 16:7755:0.50000
-1 1:9551:0.3651 2:610:0.3651 3:9952:0.3651 4:9619:0.3651 5:3780:0.3651 6:185:0.3651 7:7424:0.3651 8:4515:0.3651 9:3899:0.3651 10:4465:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:8926:0.50000 16:4511:0.50000
-1 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:9115:0.3651 6:185:0.3651 7:2569:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:3780:0.3651 6:185:0.3651 7:5671:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:4222:0.3651 2:1163:0.3651 3:8672:0.3651 4:2183:0.3651 5:9653:0.3651 6:185:0.3651 7:5671:0.3651 8:8131:0.3651 9:3899:0.3651 10:215:0.3651 11:1520:0.3651 12:7685:0.3651 13:2738:0.3651 14:2935:0.3651 15:6145:0.3651 17:2434:0.50000 16:7755:0.50000
-0 1:9551:0.3651 2:610:0.3651 3:9952:0.3651 4:9619:0.3651 5:7389:0.3651 6:185:0.3651 7:8644:0.3651 8:2285:0.3651 9:4861:0.3651 10:6908:0.3651 11:1520:0.3651 12:9714:0.3651 13:2738:0.3651 14:2935:0.3651 15:5428:0.3651 17:8926:0.50000 16:4511:0.50000

http://git-wip-us.apache.org/repos/asf/incubator-hivemall/blob/391e7f1c/core/src/test/resources/hivemall/fm/bigdata.tr.txt.gz
----------------------------------------------------------------------
diff --git a/core/src/test/resources/hivemall/fm/bigdata.tr.txt.gz b/core/src/test/resources/hivemall/fm/bigdata.tr.txt.gz
new file mode 100644
index 0000000..bd4cbae
Binary files /dev/null and b/core/src/test/resources/hivemall/fm/bigdata.tr.txt.gz differ