Merge pull request #23 from apache/WAYANG-36

[Wayang-36] Adding benchmarks
diff --git a/README.md b/README.md
index 11a92ba..15b719c 100644
--- a/README.md
+++ b/README.md
@@ -135,10 +135,6 @@
 This app will try to find appropriate values for the question marks (`?`) in the load profile estimator templates to fit the gathered execution data and ready-made configuration entries for the load profile estimators.
 You can then copy them into your configuration.
 
-## Examples
-
-For some executable examples, have a look at [this repository](https://github.com/sekruse/rheem-examples).
-
 ### WordCount
 
 The "Hello World!" of data processing systems is the wordcount.
diff --git a/pom.xml b/pom.xml
index 5bad3e0..37ba3ec 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1083,18 +1083,15 @@
                         <exclude>**/*.iml</exclude>
 
                         <exclude>**/*.yaml</exclude>
+                        <exclude>**/*.json</exclude>
 
                         <exclude>**/*.log</exclude>
-                        <exclude>**/0-to-10000.sequence_file</exclude>
+                        <exclude>**/*.input</exclude>
 
                         <exclude>**/*.svg</exclude>
                         <exclude>**/Gemfile.lock</exclude>
                         <exclude>**/.jekyll-cache/**</exclude>
                         <exclude>**/README.md</exclude>
-                        <!-- TODO: I am pretty sure we need to remove this and replace it with a different text -->
-                        <exclude>**/src/test/resources/ulysses.txt</exclude>
-                        <exclude>**/src/test/resources/test.edgelist</exclude>
-                        <exclude>**/src/test/resources/*.txt</exclude>
                     </excludes>
                 </configuration>
             </plugin>
@@ -1305,5 +1302,6 @@
         <module>wayang-profiler</module>
         <module>wayang-plugins</module>
         <module>wayang-resources</module>
+      <module>wayang-benchmark</module>
     </modules>
 </project>
diff --git a/wayang-benchmark/README.md b/wayang-benchmark/README.md
new file mode 100644
index 0000000..ffd3fc2
--- /dev/null
+++ b/wayang-benchmark/README.md
@@ -0,0 +1,128 @@
+# Apache Wayang Benchmarks <img align="right" width="128px" src="https://wayang.apache.org/assets/img/logo/logo_400x160.png" alt="Wayang logo">
+
+This repository provides example applications and further benchmarking tools to evaluate and get started with [Apache Wayang](https://wayang.apache.org).
+
+Below we provide detailed information on our various benchmark components, including running instructions. For the configuration of Apache Wayang itself, please consult the [Apache Wayang repository](https://github.com/apache/incubator-wayang) or feel free to reach out on [dev@wayang.apache.org](mailto:dev@wayang.apache.org).
+
+## Apache Wayang Example Applications
+
+### WordCount
+
+**Description.** This app takes a text input file and counts the number occurrences of each word in the text. This simple app has become some sort of _"Hello World"_ program for data processing systems.
+
+**Running the app.** To run the app, launch the main class:
+```java
+org.apache.wayang.apps.wordcount.WordCountScala
+```
+Even though this app is written in Scala, you can launch it in a regular JVM. Run the app without parameters to get a description of the required parameters.
+
+**Datasets.** Find below a list of datasets that can be used to benchmark Apache Wayang in combination with this app:
+* [DBpedia - Long abstracts](http://wiki.dbpedia.org/Downloads2015-10) _NB: Consider stripping of the RDF container around the abstracts. It's not necessary, though._
+
+### Word2NVec
+
+**Description.** Akin to Google's [Word2Vec](https://arxiv.org/abs/1301.3781), this app creates vector representations of words from a corpus based on its neighbors. This app is a bit simpler in the sense that it calculates the average neighborhood of each word rather than determining a lower-dimensional representation. The resulting vectors can be used, e.g., to cluster words and find related terms.
+
+**Running the app.** To run the app, launch the main class:
+```java
+org.apache.wayang.apps.simwords.Word2NVec
+```
+Even though this app is written in Scala, you can launch it in a regular JVM. Run the app without parameters to get a description of the required parameters.
+
+**Datasets.** Find below a list of datasets that can be used to benchmark Apache Wayang in combination with this app:
+* [DBpedia - Long abstracts](http://wiki.dbpedia.org/Downloads2015-10) _NB: Consider stripping of the RDF container around the abstracts. It's not necessary, though._
+
+### TPC-H Query 3
+
+**Description.** This app executes a query from the established TPC-H benchmark. We provide several variants that work either on data in databases, in files, or in a mixture of both. Thus, this app requires cross-platform execution.
+
+**Running the app.** To run the app, launch the main class:
+```java
+org.apache.wayang.apps.tpch.TpcH
+```
+Even though this app is written in Scala, you can launch it in a regular JVM. Run the app without parameters to get a description of the required parameters. Note that you will have to configure Apache Wayang, such that can access the database. Furthermore, this app depends on the following configuration keys:
+* `wayang.apps.tpch.csv.customer`: URL to the `CUSTOMER` file
+* `wayang.apps.tpch.csv.orders`: URL to the `ORDERS` file
+* `wayang.apps.tpch.csv.lineitem`: URL to the `LINEITEM` file
+
+**Datasets.** The datasets for this app can be generated with the [TPC-H tools](http://www.tpc.org/tpch/). The generated datasets can then be either put into a database and/or a filesystem.
+
+### SINDY
+
+**Description.** This app provides the data profiling algorithm [SINDY](https://subs.emis.de/LNI/Proceedings/Proceedings241/article24.html) that discovers inclusion dependencies in a relational database.
+
+**Running the app.** To run the app, launch the main class:
+```java
+org.apache.wayang.apps.sindy.Sindy
+```
+Even though this app is written in Scala, you can launch it in a regular JVM. Run the app without parameters to get a description of the required parameters.
+
+**Datasets.** Find below a list of datasets that can be used to benchmark Apache Wayang in combination with this app:
+* CSV files generated with the [TPC-H tools](http://www.tpc.org/tpch/)
+* [other CSV files](https://hpi.de/naumann/projects/repeatability/data-profiling/metanome-ind-algorithms.html)
+
+### SGD
+
+**Description.** This app implements the stochastic gradient descent algorithm. SGD is an optimization algorithm that minimizes a loss function and can be used in many tasks of supervised machine learning. The current implementation uses the logistic loss and can thus, be used for classification. As many other machine learning techniques, SGD is a highly iterative algorithm.
+
+**Running the app.** To run the app, launch the main class:
+```java
+org.apache.wayang.apps.sgd.SGD
+```
+Even though this app is written in Scala, you can launch it in a regular JVM. Run the app without parameters to get a description of the required parameters.
+
+**Datasets.** Find below a list of datasets that can be used to benchmark Apache Wayang in combination with this app:
+* [HIGGS](https://archive.ics.uci.edu/ml/datasets/HIGGS)
+* [Other datasets](http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/binary.html)
+
+### k-means
+
+**Description.** Being a well-known method to cluster data points in a Euclidian space. As many other machine learning techniques, k-means is an iterative algorithm.
+
+**Running the app.** To run the app, launch the main class:
+```java
+org.apache.wayang.apps.kmeans.Kmeans
+```
+or
+```java
+org.apache.wayang.apps.kmeans.postgres.Kmeans
+```
+The latter assumes data to reside in a filesystem, while the other assumes data to reside in PostgreSQL. For the latter case, you will have to configure Apache Wayang, such that it can access the database.
+Even though this app is written in Scala, you can launch it in a regular JVM. Run the app without parameters to get a description of the required parameters.
+
+**Datasets.** We provide a [data generator](https://github.com/apache/incubator-wayang/blob/master/wayang-benchmark/src/test/resources/kmeans-datagenerator.py) to generate files that can be clustered. You can further load these files into the database assuming the following schema:
+```sql
+CREATE TABLE "<table_name_of_your_choice>" (x float8, y float8);
+```
+
+### CrocoPR
+
+**Description.** This app implements the cross-community PageRank: It takes as input two graphs, merges them, and runs a standard PageRank on the resulting graph. The preprocessing and PageRank steps typically lend themselves to be executed on different platforms.
+
+**Running the app.** To run the app, launch the main class:
+```java
+org.apache.wayang.apps.crocopr.CrocoPR
+```
+Even though this app is written in Scala, you can launch it in a regular JVM. Run the app without parameters to get a description of the required parameters.
+
+**Datasets.** This app works on RDF files, more specifically the [Wikipedia pagelinks via DBpedia](http://wiki.dbpedia.org/Downloads2015-10). Note that this app requires two input files. For the purpose of benchmarking, it is fine to use the same input file twice.
+
+## Optimizer experiments
+
+### Optimizer scalability
+
+**Description.** This app generates Apache Wayang plans with specific predefined topologies but of arbitrary size. This allows to experimentally determine the scalability of Apache Wayang's optimizer to large plans.
+
+**Running the app.** To run the app, launch the main class:
+```java
+org.apache.wayang.apps.benchmark.OptimizerScalabilityTest
+```
+Even though this app is written in Scala, you can launch it in a regular JVM. Run the app without parameters to get a description of the required parameters. Furthermore, the following configuration can be interesting:
+- `wayang.core.optimizer.pruning.strategies`: controls the pruning strategy to be used when enumerating alternative plans
+  - admissible values: empty or comma-separated list of `org.apache.wayang.core.optimizer.enumeration.LatentOperatorPruningStrategy` (default), `org.apache.wayang.core.optimizer.enumeration.TopKPruningStrategy`, `org.apache.wayang.core.optimizer.enumeration.RandomPruningStrategy`, and `org.apache.wayang.core.optimizer.enumeration.SinglePlatformPruningStrategy` (order-sensitive)
+- `wayang.core.optimizer.pruning.topk`: controls the _k_ for the top-k pruning
+- `wayang.core.optimizer.enumeration.concatenationprio`: controls the order of the enumeration
+  - admissible values: `slots`, `plans`, `plans2`, `none`, `random`
+- `wayang.core.optimizer.enumeration.invertconcatenations` invert the above mentioned enumeration order
+  - admissible value: `false` (default), `true`
+  
diff --git a/wayang-benchmark/pom.xml b/wayang-benchmark/pom.xml
new file mode 100644
index 0000000..477c752
--- /dev/null
+++ b/wayang-benchmark/pom.xml
@@ -0,0 +1,78 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+  -->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+  xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <parent>
+    <artifactId>wayang</artifactId>
+    <groupId>org.apache.wayang</groupId>
+    <version>0.6.0-SNAPSHOT</version>
+  </parent>
+  <modelVersion>4.0.0</modelVersion>
+
+  <artifactId>wayang-benchmark</artifactId>
+  <packaging>jar</packaging>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.wayang</groupId>
+      <artifactId>wayang-core</artifactId>
+      <version>0.6.0-SNAPSHOT</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.wayang</groupId>
+      <artifactId>wayang-basic</artifactId>
+      <version>0.6.0-SNAPSHOT</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.wayang</groupId>
+      <artifactId>wayang-utils-profile-db</artifactId>
+      <version>0.6.0-SNAPSHOT</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.wayang</groupId>
+      <artifactId>wayang-api-scala-java</artifactId>
+      <version>0.6.0-SNAPSHOT</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.wayang</groupId>
+      <artifactId>wayang-java</artifactId>
+      <version>0.6.0-SNAPSHOT</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.wayang</groupId>
+      <artifactId>wayang-spark</artifactId>
+      <version>0.6.0-SNAPSHOT</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.wayang</groupId>
+      <artifactId>wayang-postgres</artifactId>
+      <version>0.6.0-SNAPSHOT</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.wayang</groupId>
+      <artifactId>wayang-sqlite3</artifactId>
+      <version>0.6.0-SNAPSHOT</version>
+    </dependency>
+  </dependencies>
+
+
+
+</project>
diff --git a/wayang-benchmark/src/main/java/org/apache/wayang/apps/sgd/SGDImpl.java b/wayang-benchmark/src/main/java/org/apache/wayang/apps/sgd/SGDImpl.java
new file mode 100644
index 0000000..dbd7eac
--- /dev/null
+++ b/wayang-benchmark/src/main/java/org/apache/wayang/apps/sgd/SGDImpl.java
@@ -0,0 +1,265 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.sgd;
+
+import org.apache.wayang.api.DataQuantaBuilder;
+import org.apache.wayang.api.JavaPlanBuilder;
+import org.apache.wayang.basic.data.Tuple2;
+import org.apache.wayang.commons.util.profiledb.model.Experiment;
+import org.apache.wayang.core.api.Configuration;
+import org.apache.wayang.core.api.WayangContext;
+import org.apache.wayang.core.function.ExecutionContext;
+import org.apache.wayang.core.function.FunctionDescriptor;
+import org.apache.wayang.core.plugin.Plugin;
+import org.apache.wayang.core.util.WayangCollections;
+import org.apache.wayang.core.util.Tuple;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * This class executes a stochastic gradient descent optimization on Apache Wayang.
+ */
+public class SGDImpl {
+
+    private final Configuration configuration;
+
+    /**
+     * {@link Plugin}s to use for the SGD.
+     */
+    private final List<Plugin> plugins;
+
+    public SGDImpl(Configuration configuration, Plugin[] plugins) {
+        this.configuration = configuration;
+        this.plugins = Arrays.asList(plugins);
+    }
+
+    public double[] apply(String datasetUrl,
+                          int datasetSize,
+                          int features,
+                          int maxIterations,
+                          double accuracy,
+                          int sampleSize) {
+        return this.apply(datasetUrl, datasetSize, features, maxIterations, accuracy, sampleSize, null);
+    }
+
+    public double[] apply(String datasetUrl,
+                          int datasetSize,
+                          int features,
+                          int maxIterations,
+                          double accuracy,
+                          int sampleSize,
+                          Experiment experiment) {
+
+        // Initialize the builder.
+        WayangContext wayangContext = new WayangContext(this.configuration);
+        for (Plugin plugin : this.plugins) {
+            wayangContext.withPlugin(plugin);
+        }
+        JavaPlanBuilder javaPlanBuilder = new JavaPlanBuilder(wayangContext);
+        if (experiment != null) javaPlanBuilder.withExperiment(experiment);
+        javaPlanBuilder.withUdfJarOf(this.getClass());
+
+        // Create initial weights.
+        List<double[]> weights = Arrays.asList(new double[features]);
+        final DataQuantaBuilder<?, double[]> weightsBuilder = javaPlanBuilder
+                .loadCollection(weights).withName("init weights");
+
+        // Load and transform the data.
+        final DataQuantaBuilder<?, double[]> transformBuilder = javaPlanBuilder
+                .readTextFile(datasetUrl).withName("source")
+                .map(new Transform(features)).withName("transform");
+
+        // Do the SGD
+        Collection<double[]> results =
+                weightsBuilder.doWhile(new LoopCondition(accuracy, maxIterations), w -> {
+                    // Sample the data and update the weights.
+                    DataQuantaBuilder<?, double[]> newWeightsDataset = transformBuilder
+                            .sample(sampleSize).withDatasetSize(datasetSize).withBroadcast(w, "weights")
+                            .map(new ComputeLogisticGradient()).withBroadcast(w, "weights").withName("compute")
+                            .reduce(new Sum()).withName("reduce")
+                            .map(new WeightsUpdate()).withBroadcast(w, "weights").withName("update");
+
+                    // Calculate the convergence criterion.
+                    DataQuantaBuilder<?, Tuple2<Double, Double>> convergenceDataset = newWeightsDataset
+                            .map(new ComputeNorm()).withBroadcast(w, "weights");
+
+                    return new Tuple<>(newWeightsDataset, convergenceDataset);
+                }).withExpectedNumberOfIterations(maxIterations).collect();
+
+        // Return the results.
+        return WayangCollections.getSingleOrNull(results); // null to support experiments with skipped execution
+    }
+}
+
+class Transform implements FunctionDescriptor.SerializableFunction<String, double[]> {
+
+    int features;
+
+    public Transform(int features) {
+        this.features = features;
+    }
+
+    @Override
+    public double[] apply(String line) {
+        String[] pointStr = line.split(" ");
+        double[] point = new double[features + 1];
+        point[0] = Double.parseDouble(pointStr[0]);
+        for (int i = 1; i < pointStr.length; i++) {
+            if (pointStr[i].equals("")) {
+                continue;
+            }
+            String kv[] = pointStr[i].split(":", 2);
+            point[Integer.parseInt(kv[0]) - 1] = Double.parseDouble(kv[1]);
+        }
+        return point;
+    }
+}
+
+class ComputeLogisticGradient implements FunctionDescriptor.ExtendedSerializableFunction<double[], double[]> {
+
+    double[] weights;
+
+    @Override
+    public double[] apply(double[] point) {
+        double[] gradient = new double[point.length];
+        double dot = 0;
+        for (int j = 0; j < weights.length; j++)
+            dot += weights[j] * point[j + 1];
+
+        for (int j = 0; j < weights.length; j++)
+            gradient[j + 1] = ((1 / (1 + Math.exp(-1 * dot))) - point[0]) * point[j + 1];
+
+        gradient[0] = 1; //counter for the step size required in the update
+
+        return gradient;
+    }
+
+    @Override
+    public void open(ExecutionContext executionContext) {
+        this.weights = (double[]) executionContext.getBroadcast("weights").iterator().next();
+    }
+}
+
+class Sum implements FunctionDescriptor.SerializableBinaryOperator<double[]> {
+
+    @Override
+    public double[] apply(double[] o, double[] o2) {
+        double[] g1 = o;
+        double[] g2 = o2;
+
+        if (g2 == null) //samples came from one partition only
+            return g1;
+
+        if (g1 == null) //samples came from one partition only
+            return g2;
+
+        double[] sum = new double[g1.length];
+        sum[0] = g1[0] + g2[0]; //count
+        for (int i = 1; i < g1.length; i++)
+            sum[i] = g1[i] + g2[i];
+
+        return sum;
+    }
+}
+
+class WeightsUpdate implements FunctionDescriptor.ExtendedSerializableFunction<double[], double[]> {
+
+    double[] weights;
+    int current_iteration;
+
+    double stepSize = 1;
+    double regulizer = 0;
+
+    public WeightsUpdate() {
+    }
+
+    public WeightsUpdate(double stepSize, double regulizer) {
+        this.stepSize = stepSize;
+        this.regulizer = regulizer;
+    }
+
+    @Override
+    public double[] apply(double[] input) {
+
+        double count = input[0];
+        double alpha = (stepSize / (current_iteration + 1));
+        double[] newWeights = new double[weights.length];
+        for (int j = 0; j < weights.length; j++) {
+            newWeights[j] = (1 - alpha * regulizer) * weights[j] - alpha * (1.0 / count) * input[j + 1];
+        }
+        return newWeights;
+    }
+
+    @Override
+    public void open(ExecutionContext executionContext) {
+        this.weights = (double[]) executionContext.getBroadcast("weights").iterator().next();
+        this.current_iteration = executionContext.getCurrentIteration();
+    }
+}
+
+class ComputeNorm implements FunctionDescriptor.ExtendedSerializableFunction<double[], Tuple2<Double, Double>> {
+
+    double[] previousWeights;
+
+    @Override
+    public Tuple2<Double, Double> apply(double[] weights) {
+        double normDiff = 0.0;
+        double normWeights = 0.0;
+        for (int j = 0; j < weights.length; j++) {
+//            normDiff += Math.sqrt(Math.pow(Math.abs(weights[j] - input[j]), 2));
+            normDiff += Math.abs(weights[j] - previousWeights[j]);
+//            normWeights += Math.sqrt(Math.pow(Math.abs(input[j]), 2));
+            normWeights += Math.abs(weights[j]);
+        }
+        return new Tuple2(normDiff, normWeights);
+    }
+
+    @Override
+    public void open(ExecutionContext executionContext) {
+        this.previousWeights = (double[]) executionContext.getBroadcast("weights").iterator().next();
+    }
+}
+
+
+class LoopCondition implements FunctionDescriptor.ExtendedSerializablePredicate<Collection<Tuple2<Double, Double>>> {
+
+    public double accuracy;
+    public int max_iterations;
+
+    private int current_iteration;
+
+    public LoopCondition(double accuracy, int max_iterations) {
+        this.accuracy = accuracy;
+        this.max_iterations = max_iterations;
+    }
+
+    @Override
+    public boolean test(Collection<Tuple2<Double, Double>> collection) {
+        Tuple2<Double, Double> input = WayangCollections.getSingle(collection);
+        System.out.println("Running iteration: " + current_iteration);
+        return (input.field0 < accuracy * Math.max(input.field1, 1.0) || current_iteration > max_iterations);
+    }
+
+    @Override
+    public void open(ExecutionContext executionContext) {
+        this.current_iteration = executionContext.getCurrentIteration();
+    }
+}
diff --git a/wayang-benchmark/src/main/java/org/apache/wayang/apps/sgd/SGDImprovedImpl.java b/wayang-benchmark/src/main/java/org/apache/wayang/apps/sgd/SGDImprovedImpl.java
new file mode 100644
index 0000000..1532f57
--- /dev/null
+++ b/wayang-benchmark/src/main/java/org/apache/wayang/apps/sgd/SGDImprovedImpl.java
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.sgd;
+
+import org.apache.wayang.commons.util.profiledb.model.Experiment;
+import org.apache.wayang.api.DataQuantaBuilder;
+import org.apache.wayang.api.JavaPlanBuilder;
+import org.apache.wayang.basic.data.Tuple2;
+import org.apache.wayang.core.api.Configuration;
+import org.apache.wayang.core.api.WayangContext;
+import org.apache.wayang.core.function.ExecutionContext;
+import org.apache.wayang.core.function.FunctionDescriptor;
+import org.apache.wayang.core.plugin.Plugin;
+import org.apache.wayang.core.util.WayangCollections;
+import org.apache.wayang.core.util.Tuple;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * This class executes a stochastic gradient descent optimization on Apache Wayang, just like {@link SGDImpl}. However,
+ * it used the {@link org.apache.wayang.basic.operators.MapPartitionsOperator} for performance improvements.
+ */
+public class SGDImprovedImpl {
+    private final Configuration configuration;
+
+    /**
+     * {@link Plugin}s to use for the SGD.
+     */
+    private final List<Plugin> plugins;
+
+    public SGDImprovedImpl(Configuration configuration, Plugin[] plugins) {
+        this.configuration = configuration;
+        this.plugins = Arrays.asList(plugins);
+    }
+
+    public double[] apply(String datasetUrl,
+                          int datasetSize,
+                          int features,
+                          int maxIterations,
+                          double accuracy,
+                          int sampleSize) {
+        return this.apply(datasetUrl, datasetSize, features, maxIterations, accuracy, sampleSize, null);
+    }
+
+    public double[] apply(String datasetUrl,
+                          int datasetSize,
+                          int features,
+                          int maxIterations,
+                          double accuracy,
+                          int sampleSize,
+                          Experiment experiment) {
+
+        // Initialize the builder.
+        WayangContext wayangContext = new WayangContext(this.configuration);
+        for (Plugin plugin : this.plugins) {
+            wayangContext.withPlugin(plugin);
+        }
+        JavaPlanBuilder javaPlanBuilder = new JavaPlanBuilder(wayangContext);
+        if (experiment != null) javaPlanBuilder.withExperiment(experiment);
+        javaPlanBuilder.withUdfJarOf(this.getClass());
+
+        // Create initial weights.
+        List<double[]> weights = Arrays.asList(new double[features]);
+        final DataQuantaBuilder<?, double[]> weightsBuilder = javaPlanBuilder
+                .loadCollection(weights).withName("init weights");
+
+        // Load and transform the data.
+        final DataQuantaBuilder<?, double[]> transformBuilder = javaPlanBuilder
+                .readTextFile(datasetUrl).withName("source")
+                .mapPartitions(new TransformPerPartition(features)).withName("transform");
+
+
+        // Do the SGD
+        Collection<double[]> results  =
+                weightsBuilder.doWhile(new LoopCondition(accuracy, maxIterations), w -> {
+                    // Sample the data and update the weights.
+                    DataQuantaBuilder<?, double[]> newWeightsDataset = transformBuilder
+                        .sample(sampleSize).withDatasetSize(datasetSize).withBroadcast(w, "weights")
+                        .mapPartitions(new ComputeLogisticGradientPerPartition(features)).withBroadcast(w, "weights").withName("compute")
+                        .reduce(new Sum()).withName("reduce")
+                        .map(new WeightsUpdate()).withBroadcast(w, "weights").withName("update");
+
+                    // Calculate the convergence criterion.
+                    DataQuantaBuilder<?, Tuple2<Double, Double>> convergenceDataset = newWeightsDataset
+                            .map(new ComputeNorm()).withBroadcast(w, "weights");
+
+                    return new Tuple<>(newWeightsDataset, convergenceDataset);
+                }).withExpectedNumberOfIterations(maxIterations).collect();
+
+        // Return the results.
+        return WayangCollections.getSingleOrNull(results); // Support null for when execution is skipped.
+
+    }
+}
+
+class TransformPerPartition implements FunctionDescriptor.SerializableFunction<Iterable<String>, Iterable<double[]>> {
+
+    int features;
+
+    public TransformPerPartition (int features) {
+        this.features = features;
+    }
+
+    @Override
+    public Iterable<double[]> apply(Iterable<String> lines) {
+        List<double[]> list = new ArrayList<>();
+        lines.forEach(line -> {
+            String[] pointStr = line.split(" ");
+            double[] point = new double[features+1];
+            point[0] = Double.parseDouble(pointStr[0]);
+            for (int i = 1; i < pointStr.length; i++) {
+                if (pointStr[i].equals("")) {
+                    continue;
+                }
+                String kv[] = pointStr[i].split(":", 2);
+                point[Integer.parseInt(kv[0])-1] = Double.parseDouble(kv[1]);
+            }
+            list.add(point);
+        });
+        return list;
+    }
+}
+
+class ComputeLogisticGradientPerPartition implements FunctionDescriptor.ExtendedSerializableFunction<Iterable<double[]>, Iterable<double[]>> {
+
+    double[] weights;
+    double[] sumGradOfPartition;
+    int features;
+
+    public ComputeLogisticGradientPerPartition(int features) {
+        this.features = features;
+        sumGradOfPartition = new double[features + 1]; //position 0 is for the count
+    }
+
+    @Override
+    public Iterable<double[]> apply(Iterable<double[]> points) {
+        List<double[]> list = new ArrayList<>(1);
+        points.forEach(point -> {
+            double dot = 0;
+            for (int j = 0; j < weights.length; j++)
+                dot += weights[j] * point[j + 1];
+            for (int j = 0; j < weights.length; j++)
+                sumGradOfPartition[j + 1] += ((1 / (1 + Math.exp(-1 * dot))) - point[0]) * point[j + 1];
+
+            sumGradOfPartition[0] += 1; //counter for the step size required in the update
+
+        });
+        list.add(sumGradOfPartition);
+        return list;
+    }
+
+    @Override
+    public void open(ExecutionContext executionContext) {
+        this.weights = (double[]) executionContext.getBroadcast("weights").iterator().next();
+        sumGradOfPartition = new double[features + 1];
+    }
+}
diff --git a/wayang-benchmark/src/main/java/org/apache/wayang/apps/tpch/Main.java b/wayang-benchmark/src/main/java/org/apache/wayang/apps/tpch/Main.java
new file mode 100644
index 0000000..c020b34
--- /dev/null
+++ b/wayang-benchmark/src/main/java/org/apache/wayang/apps/tpch/Main.java
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.tpch;
+
+import org.apache.wayang.apps.tpch.data.LineItemTuple;
+import org.apache.wayang.apps.tpch.data.q1.GroupKey;
+import org.apache.wayang.apps.tpch.data.q1.ReturnTuple;
+import org.apache.wayang.basic.operators.*;
+import org.apache.wayang.core.api.WayangContext;
+import org.apache.wayang.core.plan.wayangplan.WayangPlan;
+import org.apache.wayang.core.util.ReflectionUtils;
+import org.apache.wayang.java.Java;
+import org.apache.wayang.java.platform.JavaPlatform;
+import org.apache.wayang.spark.Spark;
+import org.apache.wayang.spark.platform.SparkPlatform;
+
+/**
+ * Main class for the TPC-H app based on Apache Wayang.
+ */
+public class Main {
+
+    /**
+     * Creates TPC-H Query 1, which is as follows:
+     * <pre>
+     * select
+     *  l_returnflag,
+     *  l_linestatus,
+     *  sum(l_quantity) as sum_qty,
+     *  sum(l_extendedprice) as sum_base_price,
+     *  sum(l_extendedprice*(1-l_discount)) as sum_disc_price,
+     *  sum(l_extendedprice*(1-l_discount)*(1+l_tax)) as sum_charge,
+     *  avg(l_quantity) as avg_qty,
+     *  avg(l_extendedprice) as avg_price,
+     *  avg(l_discount) as avg_disc,
+     *  count(*) as count_order
+     * from
+     *  lineitem
+     * where
+     *  l_shipdate <= date '1998-12-01' - interval '[DELTA]' day (3)
+     * group by
+     *  l_returnflag,
+     *  l_linestatus
+     * order by
+     *  l_returnflag,
+     *  l_linestatus;
+     * </pre>
+     *
+     * @param lineItemUrl URL to the lineitem CSV file
+     * @param delta       the {@code [DELTA]} parameter
+     * @return {@link WayangPlan} that implements the query
+     */
+    private static WayangPlan createQ1(String lineItemUrl, final int delta) {
+        // Read the lineitem table.
+        TextFileSource lineItemText = new TextFileSource(lineItemUrl, "UTF-8");
+
+        // Parse the rows.
+        MapOperator<String, LineItemTuple> parser = new MapOperator<>(
+                (line) -> new LineItemTuple.Parser().parse(line), String.class, LineItemTuple.class
+        );
+        lineItemText.connectTo(0, parser, 0);
+
+        // Filter by shipdate.
+        final int maxShipdate = LineItemTuple.Parser.parseDate("1998-12-01") - delta;
+        FilterOperator<LineItemTuple> filter = new FilterOperator<>(
+                (tuple) -> tuple.L_SHIPDATE <= maxShipdate, LineItemTuple.class
+        );
+        parser.connectTo(0, filter, 0);
+
+        // Project the queried attributes.
+        MapOperator<LineItemTuple, ReturnTuple> projection = new MapOperator<>(
+                (lineItemTuple) -> new ReturnTuple(
+                        lineItemTuple.L_RETURNFLAG,
+                        lineItemTuple.L_LINESTATUS,
+                        lineItemTuple.L_QUANTITY,
+                        lineItemTuple.L_EXTENDEDPRICE,
+                        lineItemTuple.L_EXTENDEDPRICE * (1 - lineItemTuple.L_DISCOUNT),
+                        lineItemTuple.L_EXTENDEDPRICE * (1 - lineItemTuple.L_DISCOUNT) * (1 + lineItemTuple.L_TAX),
+                        lineItemTuple.L_QUANTITY,
+                        lineItemTuple.L_EXTENDEDPRICE,
+                        lineItemTuple.L_DISCOUNT,
+                        1),
+                LineItemTuple.class,
+                ReturnTuple.class
+        );
+        filter.connectTo(0, projection, 0);
+
+        // Aggregation phase 1.
+        ReduceByOperator<ReturnTuple, GroupKey> aggregation = new ReduceByOperator<>(
+                (returnTuple) -> new GroupKey(returnTuple.L_RETURNFLAG, returnTuple.L_LINESTATUS),
+                ((t1, t2) -> {
+                    t1.SUM_QTY += t2.SUM_QTY;
+                    t1.SUM_BASE_PRICE += t2.SUM_BASE_PRICE;
+                    t1.SUM_DISC_PRICE += t2.SUM_DISC_PRICE;
+                    t1.SUM_CHARGE += t2.SUM_CHARGE;
+                    t1.AVG_QTY += t2.AVG_QTY;
+                    t1.AVG_PRICE += t2.AVG_PRICE;
+                    t1.AVG_DISC += t2.AVG_DISC;
+                    t1.COUNT_ORDER += t2.COUNT_ORDER;
+                    return t1;
+                }),
+                GroupKey.class,
+                ReturnTuple.class
+        );
+        projection.connectTo(0, aggregation, 0);
+
+        // Aggregation phase 2: complete AVG operations.
+        MapOperator<ReturnTuple, ReturnTuple> aggregationFinalization = new MapOperator<>(
+                (t -> {
+                    t.AVG_QTY /= t.COUNT_ORDER;
+                    t.AVG_PRICE /= t.COUNT_ORDER;
+                    t.AVG_DISC /= t.COUNT_ORDER;
+                    return t;
+                }),
+                ReturnTuple.class,
+                ReturnTuple.class
+        );
+        aggregation.connectTo(0, aggregationFinalization, 0);
+
+        // TODO: Implement sorting (as of now not possible with Wayang's basic operators).
+
+        // Print the results.
+        LocalCallbackSink<ReturnTuple> sink = LocalCallbackSink.createStdoutSink(ReturnTuple.class);
+        aggregationFinalization.connectTo(0, sink, 0);
+
+        return new WayangPlan(sink);
+    }
+
+    public static void main(String[] args) {
+        if (args.length == 0) {
+            System.err.print("Usage: <platform1>[,<platform2>]* <query number> <query parameters>*");
+            System.exit(1);
+        }
+
+        WayangPlan wayangPlan;
+        switch (Integer.parseInt(args[1])) {
+            case 1:
+                wayangPlan = createQ1(args[2], Integer.parseInt(args[3]));
+                break;
+            default:
+                System.err.println("Unsupported query number.");
+                System.exit(2);
+                return;
+        }
+
+        WayangContext wayangContext = new WayangContext();
+        for (String platform : args[0].split(",")) {
+            switch (platform) {
+                case "java":
+                    wayangContext.register(Java.basicPlugin());
+                    break;
+                case "spark":
+                    wayangContext.register(Spark.basicPlugin());
+                    break;
+                default:
+                    System.err.format("Unknown platform: \"%s\"\n", platform);
+                    System.exit(3);
+                    return;
+            }
+        }
+
+        wayangContext.execute(wayangPlan, ReflectionUtils.getDeclaringJar(Main.class), ReflectionUtils.getDeclaringJar(JavaPlatform.class));
+    }
+}
diff --git a/wayang-benchmark/src/main/java/org/apache/wayang/apps/tpch/data/LineItemTuple.java b/wayang-benchmark/src/main/java/org/apache/wayang/apps/tpch/data/LineItemTuple.java
new file mode 100644
index 0000000..9e1d124
--- /dev/null
+++ b/wayang-benchmark/src/main/java/org/apache/wayang/apps/tpch/data/LineItemTuple.java
@@ -0,0 +1,252 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.tpch.data;
+
+import java.io.Serializable;
+import java.util.Calendar;
+import java.util.GregorianCalendar;
+
+/**
+ * A tuple of the lineitem table.
+ * <p>Example line:</p>
+ * <pre>
+ * "3249925";"37271";"9775";"1";"9.00";"10874.43";"0.10";"0.04";"N";"O";"1998-04-19";"1998-06-17";"1998-04-21";"TAKE BACK RETURN         ";"AIR       ";"express instructions among the excuses nag"
+ * </pre>
+ */
+public class LineItemTuple implements Serializable {
+
+    /**
+     * {@code identifier}, {@code PK}
+     */
+    public long L_ORDERKEY;
+
+    /**
+     * {@code identifier}
+     */
+    public long L_PARTKEY;
+
+    /**
+     * {@code identifier}
+     */
+    public long L_SUPPKEY;
+
+    /**
+     * {@code integer}, {@code PK}
+     */
+    public int L_LINENUMBER;
+
+    /**
+     * {@code decimal}
+     */
+    public double L_QUANTITY;
+
+    /**
+     * {@code decimal}
+     */
+    public double L_EXTENDEDPRICE;
+
+    /**
+     * {@code decimal}
+     */
+    public double L_DISCOUNT;
+
+    /**
+     * {@code decimal}
+     */
+    public double L_TAX;
+
+    /**
+     * {@code fixed text, size 1}
+     */
+    public char L_RETURNFLAG;
+
+    /**
+     * {@code fixed text, size 1}
+     */
+    public char L_LINESTATUS;
+
+    /**
+     * {@code fixed text, size 1}
+     */
+    public int L_SHIPDATE;
+
+    /**
+     * {@code fixed text, size 1}
+     */
+    public int L_COMMITDATE;
+
+    /**
+     * {@code fixed text, size 1}
+     */
+    public int L_RECEIPTDATE;
+
+    /**
+     * {@code fixed text, size 25}
+     */
+    public String L_SHIPINSTRUCT;
+
+    /**
+     * {@code fixed text, size 10}
+     */
+    public String L_SHIPMODE;
+
+    /**
+     * {@code variable text, size 44}
+     */
+    public String L_COMMENT;
+
+    public LineItemTuple(long l_ORDERKEY,
+                         long l_PARTKEY,
+                         long l_SUPPKEY,
+                         int l_LINENUMBER,
+                         double l_QUANTITY,
+                         double l_EXTENDEDPRICE,
+                         double l_DISCOUNT,
+                         double l_TAX,
+                         char l_RETURNFLAG,
+                         int l_SHIPDATE,
+                         int l_COMMITDATE,
+                         int l_RECEIPTDATE,
+                         String l_SHIPINSTRUCT,
+                         String l_SHIPMODE,
+                         String l_COMMENT) {
+        this.L_ORDERKEY = l_ORDERKEY;
+        this.L_PARTKEY = l_PARTKEY;
+        this.L_SUPPKEY = l_SUPPKEY;
+        this.L_LINENUMBER = l_LINENUMBER;
+        this.L_QUANTITY = l_QUANTITY;
+        this.L_EXTENDEDPRICE = l_EXTENDEDPRICE;
+        this.L_DISCOUNT = l_DISCOUNT;
+        this.L_TAX = l_TAX;
+        this.L_RETURNFLAG = l_RETURNFLAG;
+        this.L_SHIPDATE = l_SHIPDATE;
+        this.L_COMMITDATE = l_COMMITDATE;
+        this.L_RECEIPTDATE = l_RECEIPTDATE;
+        this.L_SHIPINSTRUCT = l_SHIPINSTRUCT;
+        this.L_SHIPMODE = l_SHIPMODE;
+        this.L_COMMENT = l_COMMENT;
+    }
+
+    public LineItemTuple() {
+    }
+
+    /**
+     * Parses a {@link LineItemTuple} from a given CSV line (double quoted, comma-separated).
+     */
+    public static class Parser {
+
+        public LineItemTuple parse(String line) {
+            LineItemTuple tuple = new LineItemTuple();
+
+            int startPos = 0;
+            int endPos = line.indexOf(';', startPos);
+            tuple.L_ORDERKEY = Long.valueOf(line.substring(startPos + 1, endPos - 1));
+
+            startPos = endPos + 1;
+            endPos = line.indexOf(';', startPos);
+            tuple.L_PARTKEY = Long.valueOf(line.substring(startPos + 1, endPos - 1));
+
+            startPos = endPos + 1;
+            endPos = line.indexOf(';', startPos);
+            tuple.L_SUPPKEY = Long.valueOf(line.substring(startPos + 1, endPos - 1));
+
+            startPos = endPos + 1;
+            endPos = line.indexOf(';', startPos);
+            tuple.L_LINENUMBER = Integer.valueOf(line.substring(startPos + 1, endPos - 1));
+
+            startPos = endPos + 1;
+            endPos = line.indexOf(';', startPos);
+            tuple.L_QUANTITY = Double.valueOf(line.substring(startPos + 1, endPos - 1));
+
+            startPos = endPos + 1;
+            endPos = line.indexOf(';', startPos);
+            tuple.L_EXTENDEDPRICE = Double.valueOf(line.substring(startPos + 1, endPos - 1));
+
+            startPos = endPos + 1;
+            endPos = line.indexOf(';', startPos);
+            tuple.L_DISCOUNT = Double.valueOf(line.substring(startPos + 1, endPos - 1));
+
+            startPos = endPos + 1;
+            endPos = line.indexOf(';', startPos);
+            tuple.L_TAX = Double.valueOf(line.substring(startPos + 1, endPos - 1));
+
+            startPos = endPos + 1;
+            endPos = line.indexOf(';', startPos);
+            tuple.L_RETURNFLAG = line.charAt(startPos + 1);
+
+            startPos = endPos + 1;
+            endPos = line.indexOf(';', startPos);
+            tuple.L_LINESTATUS = line.charAt(startPos + 1);
+
+            startPos = endPos + 1;
+            endPos = line.indexOf(';', startPos);
+            tuple.L_SHIPDATE = parseDate(line.substring(startPos + 1, endPos - 1));
+
+            startPos = endPos + 1;
+            endPos = line.indexOf(';', startPos);
+            tuple.L_COMMITDATE = parseDate(line.substring(startPos + 1, endPos - 1));
+
+            startPos = endPos + 1;
+            endPos = line.indexOf(';', startPos);
+            tuple.L_RECEIPTDATE = parseDate(line.substring(startPos + 1, endPos - 1));
+
+            startPos = endPos + 1;
+            endPos = startPos - 1;
+            do {
+                endPos++;
+                endPos = line.indexOf(';', endPos);
+            } while (line.charAt(endPos - 1) != '"' || line.charAt(endPos + 1) != '"');
+            tuple.L_SHIPINSTRUCT = line.substring(startPos + 1, endPos - 1);
+
+            startPos = endPos + 1;
+            endPos = startPos - 1;
+            do {
+                endPos++;
+                endPos = line.indexOf(';', endPos);
+            } while (line.charAt(endPos - 1) != '"' || line.charAt(endPos + 1) != '"');
+            tuple.L_SHIPMODE = line.substring(startPos + 1, endPos - 1);
+
+            startPos = endPos + 1;
+            endPos = startPos - 1;
+            do {
+                endPos++;
+                endPos = line.indexOf(';', endPos);
+            } while (endPos >= 0 && (line.charAt(endPos - 1) != '"' || (endPos < line.length() - 1 && line.charAt(endPos + 1) != '"')));
+            assert endPos < 0 : String.format("Parsing error: unexpected ';' at %d. Input: %s", endPos, line);
+            endPos = line.length();
+            tuple.L_COMMENT = line.substring(startPos + 1, endPos - 1);
+
+            return tuple;
+        }
+
+        public static int parseDate(String dateString) {
+            Calendar calendar = GregorianCalendar.getInstance();
+            calendar.set(
+                    Integer.parseInt(dateString.substring(0, 4)),
+                    Integer.parseInt(dateString.substring(5, 7)) - 1,
+                    Integer.parseInt(dateString.substring(8, 10))
+            );
+            final int millisPerDay = 1000 * 60 * 60 * 24;
+            return (int) (calendar.getTimeInMillis() / millisPerDay);
+        }
+
+
+    }
+
+}
diff --git a/wayang-benchmark/src/main/java/org/apache/wayang/apps/tpch/data/q1/GroupKey.java b/wayang-benchmark/src/main/java/org/apache/wayang/apps/tpch/data/q1/GroupKey.java
new file mode 100644
index 0000000..db56ff9
--- /dev/null
+++ b/wayang-benchmark/src/main/java/org/apache/wayang/apps/tpch/data/q1/GroupKey.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.tpch.data.q1;
+
+import java.io.Serializable;
+import java.util.Objects;
+
+/**
+ * Grouping key used in Query 1.
+ */
+public class GroupKey implements Serializable {
+
+    public char L_RETURNFLAG;
+
+    public char L_LINESTATUS;
+
+    public GroupKey(char l_RETURNFLAG, char l_LINESTATUS) {
+        this.L_RETURNFLAG = l_RETURNFLAG;
+        this.L_LINESTATUS = l_LINESTATUS;
+    }
+
+    public GroupKey() {
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || this.getClass() != o.getClass()) return false;
+        GroupKey groupKey = (GroupKey) o;
+        return this.L_RETURNFLAG == groupKey.L_RETURNFLAG &&
+                this.L_LINESTATUS == groupKey.L_LINESTATUS;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(this.L_RETURNFLAG, this.L_LINESTATUS);
+    }
+}
diff --git a/wayang-benchmark/src/main/java/org/apache/wayang/apps/tpch/data/q1/ReturnTuple.java b/wayang-benchmark/src/main/java/org/apache/wayang/apps/tpch/data/q1/ReturnTuple.java
new file mode 100644
index 0000000..710609b
--- /dev/null
+++ b/wayang-benchmark/src/main/java/org/apache/wayang/apps/tpch/data/q1/ReturnTuple.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.tpch.data.q1;
+
+import java.io.Serializable;
+
+/**
+ * Tuple that is returned by Query 1.
+ */
+public class ReturnTuple implements Serializable {
+
+    public char L_RETURNFLAG;
+
+    public char L_LINESTATUS;
+
+    public double SUM_QTY;
+
+    public double SUM_BASE_PRICE;
+
+    public double SUM_DISC_PRICE;
+
+    public double SUM_CHARGE;
+
+    public double AVG_QTY;
+
+    public double AVG_PRICE;
+
+    public double AVG_DISC;
+
+    public int COUNT_ORDER;
+
+    public ReturnTuple() {
+    }
+
+    public ReturnTuple(char l_RETURNFLAG,
+                       char l_LINESTATUS,
+                       double SUM_QTY,
+                       double SUM_BASE_PRICE,
+                       double SUM_DISC_PRICE,
+                       double SUM_CHARGE,
+                       double AVG_QTY,
+                       double AVG_PRICE,
+                       double AVG_DISC,
+                       int COUNT_ORDER) {
+        this.L_RETURNFLAG = l_RETURNFLAG;
+        this.L_LINESTATUS = l_LINESTATUS;
+        this.SUM_QTY = SUM_QTY;
+        this.SUM_BASE_PRICE = SUM_BASE_PRICE;
+        this.SUM_DISC_PRICE = SUM_DISC_PRICE;
+        this.SUM_CHARGE = SUM_CHARGE;
+        this.AVG_QTY = AVG_QTY;
+        this.AVG_PRICE = AVG_PRICE;
+        this.AVG_DISC = AVG_DISC;
+        this.COUNT_ORDER = COUNT_ORDER;
+    }
+
+    @Override
+    public String toString() {
+        return "ReturnTuple{" +
+                "L_RETURNFLAG=" + this.L_RETURNFLAG +
+                ", L_LINESTATUS=" + this.L_LINESTATUS +
+                ", SUM_QTY=" + this.SUM_QTY +
+                ", SUM_BASE_PRICE=" + this.SUM_BASE_PRICE +
+                ", SUM_DISC_PRICE=" + this.SUM_DISC_PRICE +
+                ", SUM_CHARGE=" + this.SUM_CHARGE +
+                ", AVG_QTY=" + this.AVG_QTY +
+                ", AVG_PRICE=" + this.AVG_PRICE +
+                ", AVG_DISC=" + this.AVG_DISC +
+                ", COUNT_ORDER=" + this.COUNT_ORDER +
+                '}';
+    }
+}
diff --git a/wayang-benchmark/src/main/java/org/apache/wayang/apps/wordcount/Main.java b/wayang-benchmark/src/main/java/org/apache/wayang/apps/wordcount/Main.java
new file mode 100644
index 0000000..9b613cd
--- /dev/null
+++ b/wayang-benchmark/src/main/java/org/apache/wayang/apps/wordcount/Main.java
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.wordcount;
+
+import org.apache.wayang.basic.data.Tuple2;
+import org.apache.wayang.basic.operators.*;
+import org.apache.wayang.core.api.WayangContext;
+import org.apache.wayang.core.function.FlatMapDescriptor;
+import org.apache.wayang.core.function.ReduceDescriptor;
+import org.apache.wayang.core.function.TransformationDescriptor;
+import org.apache.wayang.core.optimizer.ProbabilisticDoubleInterval;
+import org.apache.wayang.core.plan.wayangplan.WayangPlan;
+import org.apache.wayang.core.types.DataSetType;
+import org.apache.wayang.core.types.DataUnitType;
+import org.apache.wayang.core.util.ReflectionUtils;
+import org.apache.wayang.java.Java;
+import org.apache.wayang.java.platform.JavaPlatform;
+import org.apache.wayang.spark.Spark;
+import org.apache.wayang.spark.platform.SparkPlatform;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
+
+/**
+ * Example Apache Wayang App that does a word count -- the Hello World of Map/Reduce-like systems.
+ */
+public class Main {
+
+    /**
+     * Creates the {@link WayangPlan} for the word count app.
+     *
+     * @param inputFileUrl the file whose words should be counted
+     */
+    public static WayangPlan createWayangPlan(String inputFileUrl, Collection<Tuple2<String, Integer>> collector) throws URISyntaxException, IOException {
+        // Assignment mode: none.
+
+        TextFileSource textFileSource = new TextFileSource(inputFileUrl);
+        textFileSource.setName("Load file");
+
+        // for each line (input) output an iterator of the words
+        FlatMapOperator<String, String> flatMapOperator = new FlatMapOperator<>(
+                new FlatMapDescriptor<>(line -> Arrays.asList(line.split("\\W+")),
+                        String.class,
+                        String.class,
+                        new ProbabilisticDoubleInterval(100, 10000, 0.8)
+                )
+        );
+        flatMapOperator.setName("Split words");
+
+        FilterOperator<String> filterOperator = new FilterOperator<>(str -> !str.isEmpty(), String.class);
+        filterOperator.setName("Filter empty words");
+
+
+        // for each word transform it to lowercase and output a key-value pair (word, 1)
+        MapOperator<String, Tuple2<String, Integer>> mapOperator = new MapOperator<>(
+                new TransformationDescriptor<>(word -> new Tuple2<>(word.toLowerCase(), 1),
+                        DataUnitType.createBasic(String.class),
+                        DataUnitType.createBasicUnchecked(Tuple2.class)
+                ), DataSetType.createDefault(String.class),
+                DataSetType.createDefaultUnchecked(Tuple2.class)
+        );
+        mapOperator.setName("To lower case, add counter");
+
+
+        // groupby the key (word) and add up the values (frequency)
+        ReduceByOperator<Tuple2<String, Integer>, String> reduceByOperator = new ReduceByOperator<>(
+                new TransformationDescriptor<>(pair -> pair.field0,
+                        DataUnitType.createBasicUnchecked(Tuple2.class),
+                        DataUnitType.createBasic(String.class)), new ReduceDescriptor<>(
+                ((a, b) -> {
+                    a.field1 += b.field1;
+                    return a;
+                }), DataUnitType.createGroupedUnchecked(Tuple2.class),
+                DataUnitType.createBasicUnchecked(Tuple2.class)
+        ), DataSetType.createDefaultUnchecked(Tuple2.class)
+        );
+        reduceByOperator.setName("Add counters");
+
+
+        // write results to a sink
+        LocalCallbackSink<Tuple2<String, Integer>> sink = LocalCallbackSink.createCollectingSink(
+                collector,
+                DataSetType.createDefaultUnchecked(Tuple2.class)
+        );
+        sink.setName("Collect result");
+
+        // Build Rheem plan by connecting operators
+        textFileSource.connectTo(0, flatMapOperator, 0);
+        flatMapOperator.connectTo(0, filterOperator, 0);
+        filterOperator.connectTo(0, mapOperator, 0);
+        mapOperator.connectTo(0, reduceByOperator, 0);
+        reduceByOperator.connectTo(0, sink, 0);
+
+        return new WayangPlan(sink);
+    }
+
+    public static void main(String[] args) throws IOException, URISyntaxException {
+        try {
+            if (args.length == 0) {
+                System.err.print("Usage: <platform1>[,<platform2>]* <input file URL>");
+                System.exit(1);
+            }
+
+            List<Tuple2<String, Integer>> collector = new LinkedList<>();
+            WayangPlan wayangPlan = createWayangPlan(args[1], collector);
+
+            WayangContext wayangContext = new WayangContext();
+            for (String platform : args[0].split(",")) {
+                switch (platform) {
+                    case "java":
+                        wayangContext.register(Java.basicPlugin());
+                        break;
+                    case "spark":
+                        wayangContext.register(Spark.basicPlugin());
+                        break;
+                    default:
+                        System.err.format("Unknown platform: \"%s\"\n", platform);
+                        System.exit(3);
+                        return;
+                }
+            }
+
+            wayangContext.execute(wayangPlan, ReflectionUtils.getDeclaringJar(Main.class), ReflectionUtils.getDeclaringJar(JavaPlatform.class));
+
+            collector.sort((t1, t2) -> Integer.compare(t2.field1, t1.field1));
+            System.out.printf("Found %d words:\n", collector.size());
+            collector.forEach(wc -> System.out.printf("%dx %s\n", wc.field1, wc.field0));
+        } catch (Exception e) {
+            System.err.println("App failed.");
+            e.printStackTrace();
+            System.exit(4);
+        }
+    }
+
+}
diff --git a/wayang-benchmark/src/main/resources/simplelogger.properties b/wayang-benchmark/src/main/resources/simplelogger.properties
new file mode 100644
index 0000000..1eeeb50
--- /dev/null
+++ b/wayang-benchmark/src/main/resources/simplelogger.properties
@@ -0,0 +1,21 @@
+#
+# 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.
+#
+
+org.slf4j.simpleLogger.logFile = System.out
+org.slf4j.simpleLogger.defaultLogLevel = warn
+org.slf4j.simpleLogger.log.org.apache.wayang.core.api = info
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/benchmark/OptimizerScalabilityTest.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/benchmark/OptimizerScalabilityTest.scala
new file mode 100644
index 0000000..f1c598a
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/benchmark/OptimizerScalabilityTest.scala
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.benchmark
+
+import org.apache.wayang.apps.util.{ExperimentDescriptor, Parameters, ProfileDBHelper}
+import org.apache.wayang.apps.util.ProfileDBHelper
+import org.apache.wayang.core.api.{Configuration, WayangContext}
+
+/**
+  * This app provides scalability tests for Wayang's optimizer.
+  */
+object OptimizerScalabilityTest extends ExperimentDescriptor {
+
+  /**
+    * Creates a [[WayangContext]] that skips execution.
+    */
+  def createWayangContext(platformParameter: String) = {
+    val config = new Configuration
+    config.getProperties.set("wayang.core.debug.skipexecution", "true")
+    val wayangContext = new WayangContext(config)
+    Parameters.loadPlugins(platformParameter).foreach(wayangContext.register)
+    wayangContext
+  }
+
+  override def version = "0.1.0"
+
+  /**
+    * Main method to run this app.
+    */
+  def main(args: Array[String]): Unit = {
+    if (args.isEmpty) {
+      println("Usage: <experiment descriptor> <plugins> <plan type> <plan type args>...")
+      println("Plan types:")
+      println(" pipeline <pipeline length>")
+      println(" fanout <fanout degree>")
+      println(" tree <tree height>")
+    }
+
+    // Create the experiment.
+    val experiment = Parameters.createExperiment(args(0), this)
+
+    // Initialize the wayangContext.
+    val wayangContext = createWayangContext(args(1))
+    experiment.getSubject.addConfiguration("plugins", args(1))
+
+    // Create the planGenerator.
+    val planType = args(2)
+    experiment.getSubject.addConfiguration("planType", planType)
+    val planGenerator: PlanGenerator = planType match {
+      case "pipeline" => new PipelinePlanGenerator(args(3).toInt)
+      case "fanout" => new FanoutPlanGenerator(args(3).toInt)
+      case "tree" => new TreePlanGenerator(args(3).toInt)
+      case _ => sys.error(s"Unknown plan type: $planType")
+    }
+
+    // Generate and execute the plan.
+    val planTrigger = planGenerator.generatePlanTrigger(wayangContext, experiment)
+    planTrigger()
+
+    // Store the experiment.
+    ProfileDBHelper.store(experiment, wayangContext.getConfiguration)
+  }
+}
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/benchmark/PlanGenerator.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/benchmark/PlanGenerator.scala
new file mode 100644
index 0000000..7aaccbe
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/benchmark/PlanGenerator.scala
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.benchmark
+
+import org.apache.wayang.commons.util.profiledb.model.Experiment
+import org.apache.wayang.api.{DataQuanta, PlanBuilder}
+import org.apache.wayang.core.api.WayangContext
+import org.apache.wayang.core.plan.wayangplan.WayangPlan
+
+/**
+  * Utility to create random [[WayangPlan]]s.
+  */
+abstract class PlanGenerator {
+
+  /**
+    * Generate a [[WayangPlan]].
+    *
+    * @param wayangCtx that should be used to generate the [[WayangPlan]]
+    * @return a function that upon invocation triggers the execution of the [[WayangPlan]]
+    */
+  def generatePlanTrigger(wayangCtx: WayangContext, experiment: Experiment): () => Unit
+
+}
+
+class PipelinePlanGenerator(numOperators: Int) extends PlanGenerator {
+
+  require(numOperators >= 2, "At least 2 operators are required for a pipeline.")
+
+  override def generatePlanTrigger(wayangCtx: WayangContext, experiment: Experiment): () => Unit = {
+    val planBuilder = new PlanBuilder(wayangCtx, s"Generated pipeline ($numOperators OPs)")
+      .withExperiment(experiment)
+    val source = planBuilder.loadCollection(Seq(1))
+    var edge: DataQuanta[Int] = source.withName("source")
+    for (i <- 2 until numOperators) edge = edge.map(i => i).withName(s"intermediate ${i - 1}")
+    () => edge.collect()
+  }
+
+}
+
+
+class FanoutPlanGenerator(fanDegree: Int) extends PlanGenerator {
+
+  require(fanDegree >= 1, "A fan degree must be >=1.")
+
+  override def generatePlanTrigger(wayangCtx: WayangContext, experiment: Experiment): () => Unit = {
+    val planBuilder = new PlanBuilder(wayangCtx, s"Generated fanout (degree $fanDegree)")
+      .withExperiment(experiment)
+    val source = planBuilder.loadCollection(Seq(1)).withName("source")
+    var edge: DataQuanta[Int] = source.map(i => i).withName("intermediate 1")
+    for (i <- 2 to fanDegree) edge = edge.map(i => i).withName(s"intermediate $i").withBroadcast(source, "source")
+    () => edge.collect()
+  }
+
+}
+
+
+
+class TreePlanGenerator(treeHeight: Int) extends PlanGenerator {
+
+  require(treeHeight >= 1, "The tree height must be >=1.")
+
+
+  override def generatePlanTrigger(wayangCtx: WayangContext, experiment: Experiment): () => Unit = {
+    val planBuilder = new PlanBuilder(wayangCtx, s"Generated tree (height $treeHeight)")
+      .withExperiment(experiment)
+    val root = createTree(planBuilder, treeHeight)
+    () => root.collect()
+  }
+
+  /**
+    * Create [[DataQuanta]] that from the root of a tree-shaped [[WayangPlan]] with the given height.
+    * @param planBuilder to create the leave operators, which are sources
+    * @param height of the tree
+    * @return the tree root [[DataQuanta]]
+    */
+  private def createTree(planBuilder: PlanBuilder, height: Int): DataQuanta[Int] =
+    if (height == 1) planBuilder.loadCollection(Seq(1))
+    else createTree(planBuilder, height - 1).union(createTree(planBuilder, height - 1))
+
+}
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/crocopr/CreateInitialPageRanks.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/crocopr/CreateInitialPageRanks.scala
new file mode 100644
index 0000000..27b8095
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/crocopr/CreateInitialPageRanks.scala
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.crocopr
+
+import org.apache.wayang.core.function.ExecutionContext
+import org.apache.wayang.core.function.FunctionDescriptor.ExtendedSerializableFunction
+import org.apache.wayang.core.util.WayangCollections
+
+/**
+  * Creates initial page ranks.
+  */
+class CreateInitialPageRanks(numPagesBroadcast: String)
+  extends ExtendedSerializableFunction[(Long, Iterable[Long]), (Long, Double)] {
+
+  private var initialRank: Double = _
+
+  override def open(executionCtx: ExecutionContext): Unit = {
+    val numPages = WayangCollections.getSingle(executionCtx.getBroadcast[Long](numPagesBroadcast))
+    this.initialRank = 1d / numPages
+  }
+
+  override def apply(in: (Long, Iterable[Long])) = (in._1, this.initialRank)
+
+}
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/crocopr/CrocoPR.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/crocopr/CrocoPR.scala
new file mode 100644
index 0000000..9a92ba5
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/crocopr/CrocoPR.scala
@@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.crocopr
+
+import org.apache.wayang.apps.util.{ExperimentDescriptor, Parameters, ProfileDBHelper, StdOut}
+import org.apache.wayang.api.graph._
+import org.apache.wayang.api.{DataQuanta, PlanBuilder}
+import org.apache.wayang.apps.util.ProfileDBHelper
+import org.apache.wayang.commons.util.profiledb.model.Experiment
+import org.apache.wayang.core.api.exception.WayangException
+import org.apache.wayang.core.api.{Configuration, WayangContext}
+import org.apache.wayang.core.plugin.Plugin
+import org.apache.wayang.core.util.fs.FileSystems
+
+/**
+  * wayang implementation of the cross-community PageRank.
+  */
+class CrocoPR(plugins: Plugin*) {
+
+  /**
+    * Executes the cross-community PageRank on the given files.
+    *
+    * @param inputUrl1 URL to the first RDF N3 file
+    * @param inputUrl2 URL to the second RDF N3 file
+    * @return the page ranks
+    */
+  def apply(inputUrl1: String, inputUrl2: String, numIterations: Int)
+           (implicit experiment: Experiment, configuration: Configuration) = {
+    // Initialize.
+    val wayangCtx = new WayangContext(configuration)
+    plugins.foreach(wayangCtx.register)
+    implicit val planBuilder = new PlanBuilder(wayangCtx)
+      .withExperiment(experiment)
+      .withJobName(s"CrocoPR ($inputUrl1, $inputUrl2, $numIterations iterations)")
+      .withUdfJarsOf(this.getClass)
+
+    // Read the input files.
+    val links1 = readLinks(inputUrl1)
+    val links2 = readLinks(inputUrl2)
+
+    // Merge the links.
+    val allLinks = links1
+      .union(links2).withName("Union links")
+      .distinct.withName("Distinct links")
+
+    // Create vertex IDs.
+    val vertexIds = allLinks
+      .flatMap(link => Seq(link._1, link._2)).withName("Flatten vertices")
+      .distinct.withName("Distinct vertices")
+      .zipWithId.withName("Add vertex IDs")
+
+
+    type VertexId = org.apache.wayang.basic.data.Tuple2[Vertex, String]
+    val edges = allLinks
+      .join[VertexId, String](_._1, vertexIds, _.field1).withName("Join source vertex IDs")
+      .map { linkAndVertexId =>
+        (linkAndVertexId.field1.field0, linkAndVertexId.field0._2)
+      }.withName("Set source vertex ID")
+      .join[VertexId, String](_._2, vertexIds, _.field1).withName("Join target vertex IDs")
+      .map(linkAndVertexId => new Edge(linkAndVertexId.field0._1, linkAndVertexId.field1.field0)).withName("Set target vertex ID")
+
+    // Run the PageRank.
+    val pageRanks = edges.pageRank(numIterations)
+
+    // Make the page ranks readable.
+    pageRanks
+      .map(identity).withName("Hotfix")
+      .join[VertexId, Long](_.field0, vertexIds, _.field0).withName("Join page ranks with vertex IDs")
+      .map(joinTuple => (joinTuple.field1.field1, joinTuple.field0.field1)).withName("Make page ranks readable")
+      .collect()
+
+  }
+
+  /**
+    * Reads and parses an input file.
+    *
+    * @param inputUrl    URL to the file
+    * @param planBuilder used to build to create wayang operators
+    * @return [[DataQuanta]] representing the parsed file
+    */
+  def readLinks(inputUrl: String)(implicit planBuilder: PlanBuilder): DataQuanta[(String, String)] = {
+    val linkPattern =
+      """<http://dbpedia.org/resource/([^>]+)>\s+<http://dbpedia.org/ontology/wikiPageWikiLink>\s+<http://dbpedia.org/resource/([^>]+)>\s+\.""".r
+
+    planBuilder
+      .readTextFile(inputUrl).withName(s"Load $inputUrl")
+      .filter(!_.startsWith("#")).withName("Filter comments")
+      .map {
+        case linkPattern(source, target) => (source, target)
+        case str => throw new WayangException(s"Cannot parse $str.")
+      }.withName("Parse lines")
+  }
+
+}
+
+/**
+  * Companion object for [[CrocoPR]].
+  */
+object CrocoPR extends ExperimentDescriptor {
+
+  override def version = "0.1.0"
+
+  def main(args: Array[String]) {
+    // Parse parameters.
+    if (args.isEmpty) {
+      sys.error(s"Usage: <main class> ${Parameters.experimentHelp} <plugin>(,<plugin>)* <input URL1> <input URL2> <#iterations>")
+      sys.exit(1)
+    }
+    implicit val configuration = new Configuration
+    implicit val experiment = Parameters.createExperiment(args(0), this)
+    val plugins = Parameters.loadPlugins(args(1))
+    experiment.getSubject.addConfiguration("plugins", args(1))
+    val inputUrl1 = args(2)
+    experiment.getSubject.addConfiguration("input1", inputUrl1)
+    val inputUrl2 = args(3)
+    experiment.getSubject.addConfiguration("input2", inputUrl2)
+    val numIterations = args(4).toInt
+    experiment.getSubject.addConfiguration("iterations", numIterations)
+
+    // Prepare the PageRank.
+    val pageRank = new CrocoPR(plugins: _*)
+
+    // Run the PageRank.
+    val pageRanks = pageRank(inputUrl1, inputUrl2, numIterations).toSeq.sortBy(-_._2)
+
+    // Store experiment data.
+    val inputFileSize1 = FileSystems.getFileSize(inputUrl1)
+    if (inputFileSize1.isPresent) experiment.getSubject.addConfiguration("inputSize1", inputFileSize1.getAsLong)
+    val inputFileSize2 = FileSystems.getFileSize(inputUrl2)
+    if (inputFileSize2.isPresent) experiment.getSubject.addConfiguration("inputSize2", inputFileSize2.getAsLong)
+    ProfileDBHelper.store(experiment, configuration)
+
+    // Print the result.
+    println(s"Calculated ${pageRanks.size} page ranks:")
+    StdOut.printLimited(pageRanks, formatter = (pr: (String, java.lang.Float)) => s"${pr._1} has a page rank of ${pr._2}")
+  }
+
+}
+
+
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/kmeans/Kmeans.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/kmeans/Kmeans.scala
new file mode 100644
index 0000000..5a1f1fc
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/kmeans/Kmeans.scala
@@ -0,0 +1,258 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.kmeans
+
+import org.apache.wayang.apps.util.{ExperimentDescriptor, Parameters, ProfileDBHelper}
+
+import java.util
+import org.apache.wayang.api._
+import org.apache.wayang.apps.util.ProfileDBHelper
+import org.apache.wayang.commons.util.profiledb.model.Experiment
+import org.apache.wayang.core.api.{Configuration, WayangContext}
+import org.apache.wayang.core.function.ExecutionContext
+import org.apache.wayang.core.function.FunctionDescriptor.ExtendedSerializableFunction
+import org.apache.wayang.core.optimizer.costs.LoadProfileEstimators
+import org.apache.wayang.core.plugin.Plugin
+import org.apache.wayang.core.util.fs.FileSystems
+
+import scala.collection.JavaConversions._
+import scala.util.Random
+
+/**
+  * K-Means app for Apache Wayang.
+  * <p>Note the UDF load property `wayang.apps.kmeans.udfs.select-centroid.load`.</p>
+  */
+class Kmeans(plugin: Plugin*) {
+
+  def apply(k: Int, inputFile: String, iterations: Int = 20, isResurrect: Boolean = true)
+           (implicit experiment: Experiment, configuration: Configuration): Iterable[Point] = {
+    // Set up the WayangContext.
+    implicit val wayangCtx = new WayangContext(configuration)
+    plugin.foreach(wayangCtx.register)
+    val planBuilder = new PlanBuilder(wayangCtx)
+      .withJobName(s"k-means ($inputFile, k=$k, $iterations iterations)")
+      .withExperiment(experiment)
+      .withUdfJarsOf(this.getClass)
+
+    // Read and parse the input file(s).
+    val points = planBuilder
+      .readTextFile(inputFile).withName("Read file")
+      .map { line =>
+        val fields = line.split(",")
+        Point(fields(0).toDouble, fields(1).toDouble)
+      }.withName("Create points")
+
+    // Create initial centroids.
+    val initialCentroids = planBuilder
+      .loadCollection(Kmeans.createRandomCentroids(k)).withName("Load random centroids")
+
+    // Do the k-means loop.
+    val finalCentroids = initialCentroids.repeat(iterations, { currentCentroids =>
+      val newCentroids = points
+        .mapJava(
+          new SelectNearestCentroid,
+          udfLoad = LoadProfileEstimators.createFromSpecification("wayang.apps.kmeans.udfs.select-centroid.load", configuration)
+        )
+        .withBroadcast(currentCentroids, "centroids").withName("Find nearest centroid")
+        .reduceByKey(_.centroidId, _ + _).withName("Add up points")
+        .withCardinalityEstimator(k)
+        .map(_.average).withName("Average points")
+
+
+      if (isResurrect) {
+        // Resurrect "lost" centroids (that have not been nearest to ANY point).
+        val _k = k
+        val resurrectedCentroids = newCentroids
+          .map(centroid => 1).withName("Count centroids (a)")
+          .reduce(_ + _).withName("Count centroids (b)")
+          .flatMap(num => {
+            if (num < _k) println(s"Resurrecting ${_k - num} point(s).")
+            Kmeans.createRandomCentroids(_k - num)
+          }).withName("Resurrect centroids")
+        newCentroids.union(resurrectedCentroids).withName("New+resurrected centroids").withCardinalityEstimator(k)
+      } else newCentroids
+    }).withName("Loop")
+
+    // Collect the result.
+    finalCentroids
+      .map(_.toPoint).withName("Strip centroid names")
+      .collect()
+  }
+
+
+}
+
+/**
+  * Companion object of [[Kmeans]].
+  */
+object Kmeans extends ExperimentDescriptor {
+
+  override def version = "0.1.0"
+
+  def main(args: Array[String]): Unit = {
+    // Parse args.
+    if (args.length == 0) {
+      println(s"Usage: scala <main class> ${Parameters.experimentHelp} <plugin(,plugin)*> <point file> <k> <#iterations>")
+      sys.exit(1)
+    }
+
+    implicit val experiment = Parameters.createExperiment(args(0), this)
+    implicit val configuration = new Configuration
+    val plugins = Parameters.loadPlugins(args(1))
+    experiment.getSubject.addConfiguration("plugins", args(1))
+    val file = args(2)
+    experiment.getSubject.addConfiguration("input", args(2))
+    val k = args(3).toInt
+    experiment.getSubject.addConfiguration("k", args(3))
+    val numIterations = args(4).toInt
+    experiment.getSubject.addConfiguration("iterations", args(4))
+
+    // Initialize k-means.
+    val kmeans = new Kmeans(plugins: _*)
+
+    // Run k-means.
+    val centroids = kmeans(k, file, numIterations)
+
+    // Store experiment data.
+    val fileSize = FileSystems.getFileSize(file)
+    if (fileSize.isPresent) experiment.getSubject.addConfiguration("inputSize", fileSize.getAsLong)
+    ProfileDBHelper.store(experiment, configuration)
+
+    // Print the result.
+    println(s"Found ${centroids.size} centroids:")
+
+  }
+
+  /**
+    * Creates random centroids.
+    *
+    * @param n      the number of centroids to create
+    * @param random used to draw random coordinates
+    * @return the centroids
+    */
+  def createRandomCentroids(n: Int, random: Random = new Random()) =
+  // TODO: The random cluster ID makes collisions during resurrection less likely but in general permits ID collisions.
+  for (i <- 1 to n) yield TaggedPoint(random.nextGaussian(), random.nextGaussian(), random.nextInt())
+
+}
+
+/**
+  * UDF to select the closest centroid for a given [[Point]].
+  */
+class SelectNearestCentroid extends ExtendedSerializableFunction[Point, TaggedPointCounter] {
+
+  /** Keeps the broadcasted centroids. */
+  var centroids: util.Collection[TaggedPoint] = _
+
+  override def open(executionCtx: ExecutionContext) = {
+    centroids = executionCtx.getBroadcast[TaggedPoint]("centroids")
+  }
+
+  override def apply(point: Point): TaggedPointCounter = {
+    var minDistance = Double.PositiveInfinity
+    var nearestCentroidId = -1
+    for (centroid <- centroids) {
+      val distance = point.distanceTo(centroid)
+      if (distance < minDistance) {
+        minDistance = distance
+        nearestCentroidId = centroid.centroidId
+      }
+    }
+    new TaggedPointCounter(point, nearestCentroidId, 1)
+  }
+}
+
+
+/**
+  * Represents objects with an x and a y coordinate.
+  */
+sealed trait PointLike {
+
+  /**
+    * @return the x coordinate
+    */
+  def x: Double
+
+  /**
+    * @return the y coordinate
+    */
+  def y: Double
+
+}
+
+/**
+  * Represents a two-dimensional point.
+  *
+  * @param x the x coordinate
+  * @param y the y coordinate
+  */
+case class Point(x: Double, y: Double) extends PointLike {
+
+  /**
+    * Calculates the Euclidean distance to another [[Point]].
+    *
+    * @param that the other [[PointLike]]
+    * @return the Euclidean distance
+    */
+  def distanceTo(that: PointLike) = {
+    val dx = this.x - that.x
+    val dy = this.y - that.y
+    math.sqrt(dx * dx + dy * dy)
+  }
+
+  override def toString: String = f"($x%.2f, $y%.2f)"
+}
+
+/**
+  * Represents a two-dimensional point with a centroid ID attached.
+  */
+case class TaggedPoint(x: Double, y: Double, centroidId: Int) extends PointLike {
+
+  /**
+    * Creates a [[Point]] from this instance.
+    *
+    * @return the [[Point]]
+    */
+  def toPoint = Point(x, y)
+
+}
+
+/**
+  * Represents a two-dimensional point with a centroid ID and a counter attached.
+  */
+case class TaggedPointCounter(x: Double, y: Double, centroidId: Int, count: Int = 1) extends PointLike {
+
+  def this(point: PointLike, centroidId: Int, count: Int) = this(point.x, point.y, centroidId, count)
+
+  /**
+    * Adds coordinates and counts of two instances.
+    *
+    * @param that the other instance
+    * @return the sum
+    */
+  def +(that: TaggedPointCounter) = TaggedPointCounter(this.x + that.x, this.y + that.y, this.centroidId, this.count + that.count)
+
+  /**
+    * Calculates the average of all added instances.
+    *
+    * @return a [[TaggedPoint]] reflecting the average
+    */
+  def average = TaggedPoint(x / count, y / count, centroidId)
+
+}
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/kmeans/postgres/Kmeans.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/kmeans/postgres/Kmeans.scala
new file mode 100644
index 0000000..301d953
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/kmeans/postgres/Kmeans.scala
@@ -0,0 +1,258 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.kmeans.postgres
+
+import org.apache.wayang.apps.util.{ExperimentDescriptor, Parameters, ProfileDBHelper}
+
+import java.util
+import org.apache.wayang.api.{PlanBuilder, _}
+import org.apache.wayang.apps.util.ProfileDBHelper
+import org.apache.wayang.commons.util.profiledb.model.Experiment
+import org.apache.wayang.core.api.{Configuration, WayangContext}
+import org.apache.wayang.core.function.ExecutionContext
+import org.apache.wayang.core.function.FunctionDescriptor.ExtendedSerializableFunction
+import org.apache.wayang.core.optimizer.costs.LoadProfileEstimators
+import org.apache.wayang.core.plugin.Plugin
+import org.apache.wayang.core.util.fs.FileSystems
+import org.apache.wayang.postgres.operators.PostgresTableSource
+
+import scala.collection.JavaConversions._
+import scala.collection.immutable.IndexedSeq
+import scala.util.Random
+
+
+/**
+  * K-Means app for Apache Wayang on PostgreSQL.
+  * <p>Note the UDF load property `wayang.apps.kmeans.udfs.select-centroid.load`.</p>
+  */
+class Kmeans(plugin: Plugin*) {
+
+  def apply(k: Int, tableName: String, iterations: Int = 20, isResurrect: Boolean = true)
+           (implicit experiment: Experiment, configuration: Configuration): Iterable[Point] = {
+    // Set up the WayangContext.
+    implicit val wayangCtx = new WayangContext(configuration)
+    plugin.foreach(wayangCtx.register)
+    val planBuilder = new PlanBuilder(wayangCtx)
+      .withJobName(s"k-means ($tableName, k=$k, $iterations iterations)")
+      .withExperiment(experiment)
+      .withUdfJarsOf(this.getClass)
+
+    // Read and parse the input file(s).
+    val points = planBuilder
+      .readTable(new PostgresTableSource(tableName, "x", "y")).withName("Read file")
+      .map(record => Point(record.getDouble(0), record.getDouble(1))).withName("Create points")
+
+    // Create initial centroids.
+    val initialCentroids = planBuilder
+      .loadCollection(Kmeans.createRandomCentroids(k)).withName("Load random centroids")
+
+    // Do the k-means loop.
+    val finalCentroids = initialCentroids.repeat(iterations, { currentCentroids =>
+      val newCentroids = points
+        .mapJava(
+          new SelectNearestCentroid,
+          udfLoad = LoadProfileEstimators.createFromSpecification("wayang.apps.kmeans.udfs.select-centroid.load", configuration)
+        )
+        .withBroadcast(currentCentroids, "centroids").withName("Find nearest centroid")
+        .reduceByKey(_.centroidId, _ + _).withName("Add up points")
+        .withCardinalityEstimator(k)
+        .map(_.average).withName("Average points")
+
+
+      if (isResurrect) {
+        // Resurrect "lost" centroids (that have not been nearest to ANY point).
+        val _k = k
+        val resurrectedCentroids = newCentroids
+          .map(centroid => 1).withName("Count centroids (a)")
+          .reduce(_ + _).withName("Count centroids (b)")
+          .flatMap(num => {
+            if (num < _k) println(s"Resurrecting ${_k - num} point(s).")
+            Kmeans.createRandomCentroids(_k - num)
+          }).withName("Resurrect centroids")
+        newCentroids.union(resurrectedCentroids).withName("New+resurrected centroids").withCardinalityEstimator(k)
+      } else newCentroids
+    }).withName("Loop")
+
+    // Collect the result.
+    finalCentroids
+      .map(_.toPoint).withName("Strip centroid names")
+      .collect()
+  }
+
+
+}
+
+/**
+  * Companion object of [[Kmeans]].
+  */
+object Kmeans extends ExperimentDescriptor {
+
+  override def version = "0.1.0"
+
+  def main(args: Array[String]): Unit = {
+    // Parse args.
+    if (args.length == 0) {
+      println(s"Usage: scala <main class> ${Parameters.experimentHelp} <plugin(,plugin)*> <table> <k> <#iterations>")
+      sys.exit(1)
+    }
+
+    implicit val experiment = Parameters.createExperiment(args(0), this)
+    implicit val configuration = new Configuration
+    val plugins = Parameters.loadPlugins(args(1))
+    experiment.getSubject.addConfiguration("plugins", args(1))
+    val file = args(2)
+    experiment.getSubject.addConfiguration("input", args(2))
+    val k = args(3).toInt
+    experiment.getSubject.addConfiguration("k", args(3))
+    val numIterations = args(4).toInt
+    experiment.getSubject.addConfiguration("iterations", args(4))
+
+    // Initialize k-means.
+    val kmeans = new Kmeans(plugins: _*)
+
+    // Run k-means.
+    val centroids = kmeans(k, file, numIterations)
+
+    // Store experiment data.
+    val fileSize = FileSystems.getFileSize(file)
+    if (fileSize.isPresent) experiment.getSubject.addConfiguration("inputSize", fileSize.getAsLong)
+    ProfileDBHelper.store(experiment, configuration)
+
+    // Print the result.
+    println(s"Found ${centroids.size} centroids:")
+
+  }
+
+  /**
+    * Creates random centroids.
+    *
+    * @param n      the number of centroids to create
+    * @param random used to draw random coordinates
+    * @return the centroids
+    */
+  def createRandomCentroids(n: Int, random: Random = new Random()): IndexedSeq[TaggedPoint] =
+  // TODO: The random cluster ID makes collisions during resurrection less likely but in general permits ID collisions.
+    for (i <- 1 to n) yield TaggedPoint(random.nextGaussian(), random.nextGaussian(), random.nextInt())
+
+}
+
+/**
+  * UDF to select the closest centroid for a given [[Point]].
+  */
+class SelectNearestCentroid extends ExtendedSerializableFunction[Point, TaggedPointCounter] {
+
+  /** Keeps the broadcasted centroids. */
+  var centroids: util.Collection[TaggedPoint] = _
+
+  override def open(executionCtx: ExecutionContext) = {
+    centroids = executionCtx.getBroadcast[TaggedPoint]("centroids")
+  }
+
+  override def apply(point: Point): TaggedPointCounter = {
+    var minDistance = Double.PositiveInfinity
+    var nearestCentroidId = -1
+    for (centroid <- centroids) {
+      val distance = point.distanceTo(centroid)
+      if (distance < minDistance) {
+        minDistance = distance
+        nearestCentroidId = centroid.centroidId
+      }
+    }
+    new TaggedPointCounter(point, nearestCentroidId, 1)
+  }
+}
+
+
+/**
+  * Represents objects with an x and a y coordinate.
+  */
+sealed trait PointLike {
+
+  /**
+    * @return the x coordinate
+    */
+  def x: Double
+
+  /**
+    * @return the y coordinate
+    */
+  def y: Double
+
+}
+
+/**
+  * Represents a two-dimensional point.
+  *
+  * @param x the x coordinate
+  * @param y the y coordinate
+  */
+case class Point(x: Double, y: Double) extends PointLike {
+
+  /**
+    * Calculates the Euclidean distance to another [[Point]].
+    *
+    * @param that the other [[PointLike]]
+    * @return the Euclidean distance
+    */
+  def distanceTo(that: PointLike) = {
+    val dx = this.x - that.x
+    val dy = this.y - that.y
+    math.sqrt(dx * dx + dy * dy)
+  }
+
+  override def toString: String = f"($x%.2f, $y%.2f)"
+}
+
+/**
+  * Represents a two-dimensional point with a centroid ID attached.
+  */
+case class TaggedPoint(x: Double, y: Double, centroidId: Int) extends PointLike {
+
+  /**
+    * Creates a [[Point]] from this instance.
+    *
+    * @return the [[Point]]
+    */
+  def toPoint = Point(x, y)
+
+}
+
+/**
+  * Represents a two-dimensional point with a centroid ID and a counter attached.
+  */
+case class TaggedPointCounter(x: Double, y: Double, centroidId: Int, count: Int = 1) extends PointLike {
+
+  def this(point: PointLike, centroidId: Int, count: Int) = this(point.x, point.y, centroidId, count)
+
+  /**
+    * Adds coordinates and counts of two instances.
+    *
+    * @param that the other instance
+    * @return the sum
+    */
+  def +(that: TaggedPointCounter) = TaggedPointCounter(this.x + that.x, this.y + that.y, this.centroidId, this.count + that.count)
+
+  /**
+    * Calculates the average of all added instances.
+    *
+    * @return a [[TaggedPoint]] reflecting the average
+    */
+  def average = TaggedPoint(x / count, y / count, centroidId)
+
+}
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/sgd/SGD.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/sgd/SGD.scala
new file mode 100644
index 0000000..981cf1d
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/sgd/SGD.scala
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.sgd
+
+import org.apache.wayang.apps.util.{ExperimentDescriptor, Parameters, ProfileDBHelper}
+import org.apache.wayang.apps.util.ProfileDBHelper
+import org.apache.wayang.core.api.Configuration
+
+/**
+  * Companion for the [[SGDImpl]] class.
+  */
+object SGD extends ExperimentDescriptor {
+
+  override def version = "1.0"
+
+  def main(args: Array[String]): Unit = {
+    // Parse args.
+    if (args.isEmpty) {
+      println(s"Usage: scala <main class> ${Parameters.experimentHelp} <plugin(,plugin)*> " +
+        s"<aggregation (regular|preaggregation)> <dataset URL> <dataset size> <#features> <max iterations> <accuracy> <sample size>")
+      sys.exit(1)
+    }
+
+    implicit val experiment = Parameters.createExperiment(args(0), this)
+    implicit val configuration = new Configuration
+    val plugins = Parameters.loadPlugins(args(1))
+    experiment.getSubject.addConfiguration("plugins", args(1))
+    val aggregationType = args(2)
+    experiment.getSubject.addConfiguration("aggregationType", aggregationType)
+    val datasetUrl = args(3)
+    experiment.getSubject.addConfiguration("input", datasetUrl)
+    val datasetSize = args(4).toInt
+    experiment.getSubject.addConfiguration("inputSize", datasetSize)
+    val numFeatures = args(5).toInt
+    experiment.getSubject.addConfiguration("features", numFeatures)
+    val maxIterations = args(6).toInt
+    experiment.getSubject.addConfiguration("maxIterations", maxIterations)
+    val accuracy = args(7).toDouble
+    experiment.getSubject.addConfiguration("accuracy", accuracy)
+    val sampleSize = args(8).toInt
+    experiment.getSubject.addConfiguration("sampleSize", sampleSize)
+
+    var weights: Array[Double] = null
+    aggregationType match {
+      case "regular" =>
+        // Initialize the SGD algorithm.
+        val sgd = new SGDImpl(configuration, plugins.toArray)
+        // Run the SGD.
+        weights = sgd(datasetUrl, datasetSize, numFeatures, maxIterations, accuracy, sampleSize, experiment)
+      case "preaggregation" =>
+        // Initialize the SGD algorithm.
+        val sgd = new SGDImprovedImpl(configuration, plugins.toArray)
+        // Run the SGD.
+        weights = sgd(datasetUrl, datasetSize, numFeatures, maxIterations, accuracy, sampleSize, experiment)
+      case other => sys.error("Unknown aggregation type: " + other)
+    }
+
+    // Store experiment data.
+    ProfileDBHelper.store(experiment, configuration)
+
+    // Print the result.
+    if (weights != null) println(s"Determined weights: ${weights.map(w => f"$w%,.5f").mkString(", ")}")
+  }
+
+}
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/simwords/AddIdFunction.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/simwords/AddIdFunction.scala
new file mode 100644
index 0000000..4a24bcc
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/simwords/AddIdFunction.scala
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.simwords
+
+import org.apache.wayang.core.function.FunctionDescriptor.SerializableFunction
+
+import scala.util.Random
+
+/**
+  * Attaches [[String]]s with a random [[Int]].
+  */
+class AddIdFunction extends SerializableFunction[String, (String, Int)] {
+
+  lazy val random = new Random
+
+  override def apply(t: String): (String, Int) = (t, random.nextInt())
+
+}
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/simwords/CreateWordNeighborhoodFunction.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/simwords/CreateWordNeighborhoodFunction.scala
new file mode 100644
index 0000000..6ebddb3
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/simwords/CreateWordNeighborhoodFunction.scala
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.simwords
+
+import java.util
+
+import org.apache.wayang.core.function.ExecutionContext
+import org.apache.wayang.core.function.FunctionDescriptor.ExtendedSerializableFunction
+
+import scala.collection.JavaConversions._
+
+/**
+  * This function creates word neighborhood vectors out of a text.
+  */
+class CreateWordNeighborhoodFunction(neighborhoodReach: Int, dictionaryBroadcastName: String)
+  extends ExtendedSerializableFunction[String, java.lang.Iterable[(Int, SparseVector)]] {
+
+  private var dictionary: Map[String, Int] = _
+
+  private lazy val textScrubber = new TextScrubber
+
+  private lazy val collector = new java.util.ArrayList[String]
+
+  override def open(ctx: ExecutionContext): Unit = {
+    this.dictionary = ctx.getBroadcast[(String, Int)](dictionaryBroadcastName).toMap
+  }
+
+  override def apply(value: String): java.lang.Iterable[(Int, SparseVector)] = {
+    val result = new util.LinkedList[(Int, SparseVector)]()
+
+    this.textScrubber.splitAndScrub(value, this.collector)
+    // Make sure that there is at least one neighbor; otherwise, the resulting vector will not support cosine similarity
+    if (this.collector.size > 1) {
+      val wordIds = this.collector.map(this.dictionary.getOrElse(_, -1))
+      for (i <- wordIds.indices) {
+        val builder = new SparseVector.Builder
+        for (j <- math.max(0, i - neighborhoodReach) until i; if wordIds(j) != -1) {
+          builder.add(wordIds(j), 1)
+        }
+        for (j <- i + 1 until math.min(wordIds.size, i + neighborhoodReach + 1); if wordIds(j) != -1) {
+          builder.add(wordIds(j), 1)
+        }
+        if (!builder.isEmpty) result.add((wordIds(i), builder.build))
+      }
+      this.collector.clear()
+    }
+
+    result
+  }
+}
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/simwords/ResolveClusterFunction.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/simwords/ResolveClusterFunction.scala
new file mode 100644
index 0000000..827ce46
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/simwords/ResolveClusterFunction.scala
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.simwords
+
+import org.apache.wayang.core.function.ExecutionContext
+import org.apache.wayang.core.function.FunctionDescriptor.ExtendedSerializableFunction
+
+import scala.collection.JavaConversions._
+
+/**
+  * This function creates word neighborhood vectors out of a text.
+  */
+class ResolveClusterFunction(dictionaryBroadcastName: String)
+  extends ExtendedSerializableFunction[List[Int], List[String]] {
+
+  private var dictionary: Map[Int, String] = _
+
+  override def open(ctx: ExecutionContext): Unit = {
+    this.dictionary = ctx.getBroadcast[(String, Int)](dictionaryBroadcastName).map(_.swap).toMap
+  }
+
+  override def apply(ids: List[Int]): List[String] =
+    ids.map(id => dictionary.getOrElse(id, "???"))
+}
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/simwords/ScrubFunction.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/simwords/ScrubFunction.scala
new file mode 100644
index 0000000..a24f403
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/simwords/ScrubFunction.scala
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.simwords
+
+import java.lang.Iterable
+import java.util
+
+import org.apache.wayang.core.function.ExecutionContext
+import org.apache.wayang.core.function.FunctionDescriptor.ExtendedSerializableFunction
+
+/**
+  * UDF to split text lines and scrub the tokens.
+  */
+class ScrubFunction extends ExtendedSerializableFunction[String, java.lang.Iterable[String]] {
+
+  lazy val textScrubber = new TextScrubber
+
+  override def open(ctx: ExecutionContext): Unit = {}
+
+  override def apply(line: String): Iterable[String] = {
+    val result = new util.LinkedList[String]()
+    textScrubber.splitAndScrub(line, result)
+    result
+  }
+
+}
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/simwords/SelectNearestCentroidFunction.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/simwords/SelectNearestCentroidFunction.scala
new file mode 100644
index 0000000..c98a961
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/simwords/SelectNearestCentroidFunction.scala
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.simwords
+
+import org.apache.wayang.core.function.ExecutionContext
+import org.apache.wayang.core.function.FunctionDescriptor.ExtendedSerializableFunction
+import org.apache.wayang.core.util.WayangCollections
+import org.slf4j.LoggerFactory
+
+import scala.collection.JavaConversions._
+import scala.util.Random
+
+/**
+  * This functions keeps a set of centroids around and for each input word neighborhood vector, it assigns the closest
+  * centroid.
+  */
+class SelectNearestCentroidFunction(broadcastName: String)
+  extends ExtendedSerializableFunction[(Int, SparseVector), (Int, SparseVector, Int)] {
+
+  private lazy val logger = LoggerFactory.getLogger(getClass)
+
+  private var centroids: java.util.List[(Int, SparseVector)] = _
+
+  private lazy val random = new Random()
+
+  override def open(executionCtx: ExecutionContext): Unit = {
+    this.centroids = WayangCollections.asList(executionCtx.getBroadcast[(Int, SparseVector)](broadcastName))
+  }
+
+  override def apply(wnvector: (Int, SparseVector)): (Int, SparseVector, Int) = {
+    var maxSimilarity = -1d
+    var nearestCentroid: Int = -1
+    this.centroids.foreach { centroid =>
+      val similarity = math.abs(centroid._2 * wnvector._2)
+      if (similarity > maxSimilarity) {
+        maxSimilarity = similarity
+        nearestCentroid = centroid._1
+      }
+    }
+
+    if (nearestCentroid == -1) {
+      logger.info("Did not find a matching centroid for {}", wnvector)
+      maxSimilarity = 0
+      nearestCentroid = this.centroids.get(this.random.nextInt(this.centroids.size()))._1
+    }
+
+    (wnvector._1, wnvector._2, nearestCentroid)
+  }
+}
+
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/simwords/SimWords.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/simwords/SimWords.scala
new file mode 100644
index 0000000..f1ace56
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/simwords/SimWords.scala
@@ -0,0 +1,173 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.simwords
+
+import org.apache.wayang.apps.util.{ExperimentDescriptor, Parameters, ProfileDBHelper}
+import org.apache.wayang.api._
+import org.apache.wayang.apps.util.ProfileDBHelper
+import org.apache.wayang.commons.util.profiledb.model.Experiment
+import org.apache.wayang.core.api.{Configuration, WayangContext}
+import org.apache.wayang.core.optimizer.ProbabilisticDoubleInterval
+import org.apache.wayang.core.optimizer.costs.LoadProfileEstimators
+import org.apache.wayang.core.plugin.Plugin
+import org.apache.wayang.core.util.fs.FileSystems
+
+/**
+  * This app clusters words by their word neighborhoods in a corpus.
+  * <p>Note the UDF load properties `wayang.apps.simwords.udfs.create-neighborhood.load` and `wayang.apps.simwords.udfs.select-centroid.load`.</p>
+  */
+class SimWords(plugins: Plugin*) {
+
+  def apply(inputFile: String,
+            minWordOccurrences: Int,
+            neighborhoodReach: Int,
+            numClusters: Int,
+            numIterations: Int,
+            wordsPerLine: ProbabilisticDoubleInterval)
+           (implicit experiment: Experiment,
+            configuration: Configuration) = {
+
+    // Initialize.
+    val wayangCtx = new WayangContext(configuration)
+    plugins.foreach(wayangCtx.register)
+    val planBuilder = new PlanBuilder(wayangCtx)
+      .withJobName(
+        jobName = s"SimWords ($inputFile, reach=$neighborhoodReach, clusters=$numClusters, $numIterations iterations)"
+      ).withExperiment(experiment)
+      .withUdfJarsOf(this.getClass)
+
+    // Create the word dictionary
+    val _minWordOccurrences = minWordOccurrences
+    val wordIds = planBuilder
+      .readTextFile(inputFile).withName("Read corpus (1)")
+      .flatMapJava(new ScrubFunction, selectivity = wordsPerLine).withName("Split & scrub")
+      .map(word => (word, 1)).withName("Add word counter")
+      .reduceByKey(_._1, (wc1, wc2) => (wc1._1, wc1._2 + wc2._2)).withName("Sum word counters")
+      .withCardinalityEstimator((in: Long) => math.round(in * 0.01))
+      .filter(_._2 >= _minWordOccurrences, selectivity = 10d / (9d + minWordOccurrences))
+      .withName("Filter frequent words")
+      .map(_._1).withName("Strip word counter")
+      .zipWithId.withName("Zip with ID")
+      .map(t => (t.field1, t.field0.toInt)).withName("Convert ID attachment")
+
+
+    // Create the word neighborhood vectors.
+    val wordVectors = planBuilder
+      .readTextFile(inputFile).withName("Read corpus (2)")
+      .flatMapJava(
+        new CreateWordNeighborhoodFunction(neighborhoodReach, "wordIds"),
+        selectivity = wordsPerLine,
+        udfLoad = LoadProfileEstimators.createFromSpecification("wayang.apps.simwords.udfs.create-neighborhood.load", configuration)
+
+      )
+      .withBroadcast(wordIds, "wordIds")
+      .withName("Create word vectors")
+      .reduceByKey(_._1, (wv1, wv2) => (wv1._1, wv1._2 + wv2._2)).withName("Add word vectors")
+      .map { wv =>
+        wv._2.normalize(); wv
+      }.withName("Normalize word vectors")
+
+    // Generate initial centroids.
+    //    val initialCentroids = wordVectors
+    //      .customOperator[(Int, SparseVector)](
+    //      new SampleOperator[(Int, SparseVector)](numClusters, dataSetType[(Int, SparseVector)], SampleOperator.Methods.RANDOM)
+    //    ).withName("Sample centroids")
+    //      .map(x => x).withName("Identity (wa1)")
+    val _numClusters = numClusters
+    val initialCentroids = wordIds
+      .map(_._2).withName("Strip words")
+      .group().withName("Group IDs")
+      .flatMap { ids =>
+        import scala.collection.JavaConversions._
+        val idArray = ids.toArray
+        for (i <- 0 to _numClusters) yield (i, SparseVector.createRandom(idArray, .99, _numClusters))
+      }.withName("Generate centroids")
+
+    // Run k-means on the vectors.
+    val finalCentroids = initialCentroids.repeat(numIterations, { centroids: DataQuanta[(Int, SparseVector)] =>
+      val newCentroids: DataQuanta[(Int, SparseVector)] = wordVectors
+        .mapJava(
+          new SelectNearestCentroidFunction("centroids"),
+          udfLoad = LoadProfileEstimators.createFromSpecification("wayang.apps.simwords.udfs.select-centroid.load", configuration)
+        )
+        .withBroadcast(centroids, "centroids")
+        .withName("Select nearest centroids")
+        .map(assignment => (assignment._3, assignment._2)).withName("Strip word ID")
+        .reduceByKey(_._1, (wv1: (Int, SparseVector), wv2: (Int, SparseVector)) => (wv1._1, wv1._2 + wv2._2))
+        .withName("Add up cluster words").withCardinalityEstimator((in: Long) => _numClusters.toLong)
+        .map { centroid: (Int, SparseVector) => centroid._2.normalize(); centroid }.withName("Normalize centroids")
+
+      newCentroids
+    }).withName("K-means iteration").map(x => x).withName("Identity (wa2)")
+
+    // Apply the centroids to the points and resolve the word IDs.
+    val clusters = wordVectors
+      .mapJava(new SelectNearestCentroidFunction("finalCentroids")).withBroadcast(finalCentroids, "finalCentroids").withName("Select nearest final centroids")
+      .map(assigment => (assigment._3, List(assigment._1))).withName("Discard word vectors")
+      .reduceByKey(_._1, (c1, c2) => (c1._1, c1._2 ++ c2._2)).withName("Create clusters")
+      .map(_._2).withName("Discard cluster IDs")
+      .mapJava(new ResolveClusterFunction("wordIds")).withBroadcast(wordIds, "wordIds").withName("Resolve word IDs")
+
+
+    clusters.collect()
+  }
+
+}
+
+object SimWords extends ExperimentDescriptor {
+
+  override def version = "0.1.0"
+
+  def main(args: Array[String]): Unit = {
+    if (args.isEmpty) {
+      println(s"Usage: <main class> ${Parameters.experimentHelp} <plugin(,plugin)*> <input file> <min word occurrences> <neighborhood reach> <#clusters> <#iterations> [<words per line (from..to)>]")
+      sys.exit(1)
+    }
+
+    implicit val configuration = new Configuration
+    implicit val experiment = Parameters.createExperiment(args(0), this)
+    val plugins = Parameters.loadPlugins(args(1))
+    experiment.getSubject.addConfiguration("plugins", args(1))
+    val inputFile = args(2)
+    experiment.getSubject.addConfiguration("input", args(2))
+    val minWordOccurrences = args(3).toInt
+    experiment.getSubject.addConfiguration("minWordOccurrences", args(3))
+    val neighborhoodRead = args(4).toInt
+    experiment.getSubject.addConfiguration("neighborhoodReach", args(4))
+    val numClusters = args(5).toInt
+    experiment.getSubject.addConfiguration("clusters", args(5))
+    val numIterations = args(6).toInt
+    experiment.getSubject.addConfiguration("iterations", args(6))
+    val wordsPerLine = if (args.length >= 8) {
+      experiment.getSubject.addConfiguration("wordsPerLine", args(7))
+      Parameters.parseAny(args(7)).asInstanceOf[ProbabilisticDoubleInterval]
+    } else new ProbabilisticDoubleInterval(100, 10000, 0.9)
+
+    val simWords = new SimWords(plugins: _*)
+    val result = simWords(inputFile, minWordOccurrences, neighborhoodRead, numClusters, numIterations, wordsPerLine)
+
+    // Store experiment data.
+    val inputFileSize = FileSystems.getFileSize(inputFile)
+    if (inputFileSize.isPresent) experiment.getSubject.addConfiguration("inputSize", inputFileSize.getAsLong)
+    ProfileDBHelper.store(experiment, configuration)
+
+    // Print the results.
+    result.filter(_.size > 1).toIndexedSeq.sortBy(_.size).reverse.foreach(println(_))
+  }
+}
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/simwords/SparseVector.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/simwords/SparseVector.scala
new file mode 100644
index 0000000..0001ea4
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/simwords/SparseVector.scala
@@ -0,0 +1,231 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.simwords
+
+import scala.util.Random
+
+/**
+  * A [[SparseVector]] is a vector of arbitrary dimension. It does not store `0` components.
+  */
+case class SparseVector(indices: Array[Int], values: Array[Double]) {
+
+  /**
+    * Applies scalar to this vector.
+    */
+  def *=(scalar: Double): Unit = {
+    for (i <- this.values.indices) {
+      values(i) *= scalar
+    }
+  }
+
+  /**
+    * @return the length of this vector
+    */
+  def length: Double = math.sqrt(this.values.map(v => v * v).sum)
+
+  /**
+    * Keep the direction but bring to length 1.
+    */
+  def normalize(): Unit = {
+    val scalar = 1d / length
+    this *= scalar
+  }
+
+  /**
+    * Add two vectors.
+    */
+  def +(that: SparseVector): SparseVector = {
+    val newIndices: Array[Int] = new Array[Int](this.indices.length + that.indices.length)
+    val newValues: Array[Double] = new Array[Double](this.values.length + that.values.length)
+    var newI = 0
+
+    // Co-iterate the lists.
+    var (thisI, thatI) = (0, 0)
+    while (thisI < this.indices.length && thatI < that.indices.length) {
+      val (thisIndex, thatIndex) = (this.indices(thisI), that.indices(thatI))
+      if (thisIndex < thatIndex) {
+        newIndices(newI) = thisIndex
+        newValues(newI) = this.values(thisI)
+        thisI += 1
+      } else if (thisIndex > thatIndex) {
+        newIndices(newI) = thatIndex
+        newValues(newI) = that.values(thatI)
+        thatI += 1
+      } else {
+        newIndices(newI) = thisIndex
+        newValues(newI) = this.values(thisI) + that.values(thatI)
+        thisI += 1
+        thatI += 1
+      }
+      newI += 1
+    }
+
+    // Append trailing list tail.
+    while (thisI < this.indices.length) {
+      newIndices(newI) = this.indices(thisI)
+      newValues(newI) = this.values(thisI)
+      thisI += 1
+      newI += 1
+    }
+    while (thatI < that.indices.length) {
+      newIndices(newI) = that.indices(thatI)
+      newValues(newI) = that.values(thatI)
+      thatI += 1
+      newI += 1
+    }
+
+    // Build the new instance.
+    if (newI < newIndices.length) {
+      SparseVector(java.util.Arrays.copyOfRange(newIndices, 0, newI),
+        java.util.Arrays.copyOfRange(newValues, 0, newI))
+    } else {
+      SparseVector(newIndices, newValues)
+    }
+  }
+
+  /**
+    * Dot-product of two sparse vectors.
+    */
+  def *(that: SparseVector): Double = {
+    var prodSum = 0d;
+
+    // Co-iterate the lists.
+    var (thisI, thatI) = (0, 0)
+    while (thisI < this.indices.length && thatI < that.indices.length) {
+      val (thisIndex, thatIndex) = (this.indices(thisI), that.indices(thatI))
+      if (thisIndex < thatIndex) {
+        thisI += 1
+      } else if (thisIndex > thatIndex) {
+        thatI += 1
+      } else {
+        prodSum += this.values(thisI) * that.values(thatI)
+        thisI += 1
+        thatI += 1
+      }
+    }
+
+    prodSum
+  }
+
+  /**
+    * Cosine similarity of two sparse vectors.
+    */
+  def ~(that: SparseVector): Double = {
+    math.abs(this * that) / (this.length * that.length)
+  }
+
+  override def toString = s"(${
+    this.indices
+      .zip(this.values)
+      .map(component => s"${component._1}->${component._2}")
+      .mkString(", ")
+  })"
+
+  def toDictionaryString = s"{${
+    this.indices
+      .zip(this.values)
+      .map(component => s"${component._1}:${component._2}")
+      .mkString(",")
+  }}"
+
+  override def hashCode = java.util.Arrays.hashCode(this.values) ^ java.util.Arrays.hashCode(this.indices)
+
+  override def equals(o: Any): Boolean = {
+    if (o == null || !o.isInstanceOf[SparseVector]) return false;
+    val that = o.asInstanceOf[SparseVector]
+    java.util.Arrays.equals(this.indices, that.indices) && java.util.Arrays.equals(this.values, that.values)
+  }
+
+
+}
+
+/**
+  * Companion object for [[SparseVector]].
+  */
+object SparseVector {
+
+  private lazy val random = new Random
+
+  /**
+    * Create a random instance.
+    *
+    * @param indices          possible [[SparseVector#indices]]
+    * @param probCompleteness probability of using all components in `numCreations`
+    * @param numCreations     number of [[SparseVector]]s that will be created
+    */
+  def createRandom(indices: Array[Int], probCompleteness: Double, numCreations: Int, withNegative: Boolean = false) = {
+    // Find out the probability to include each component into the vector.
+    val probPickElement = calculatePickElementProb(probCompleteness, numCreations)
+
+    // Generate the components.
+    val builder = new Builder
+    for (i <- indices.indices) {
+      if (probPickElement >= this.random.nextDouble()) {
+        builder.add(indices(i), this.random.nextDouble() * (if (withNegative && this.random.nextBoolean()) -1 else 1))
+      }
+    }
+
+    // Build the vector.
+    builder.build
+  }
+
+  /**
+    * Estimate the required probability to pick an element in a single try, given a the probability of picking all
+    * elements in a number of repetitions.
+    */
+  private def calculatePickElementProb(probCompleteness: Double, numCreations: Int) =
+    1 - math.pow(1 - probCompleteness, 1d / numCreations)
+
+  class Builder {
+
+    private val components = scala.collection.mutable.Map[Int, Double]()
+
+    /**
+      * Add a delta to a component.
+      */
+    def add(dimension: Int, delta: Double): Builder = {
+      val newValue = components.getOrElse(dimension, 0d) + delta
+      if (newValue == 0) this.components.remove(dimension)
+      else this.components.update(dimension, newValue)
+      this
+    }
+
+    /**
+      * Create the [[SparseVector]].
+      */
+    def build: SparseVector = {
+      val indices = new Array[Int](this.components.size)
+      val values = new Array[Double](this.components.size)
+      var i = 0
+      for (component <- this.components.toArray.sortBy(_._1)) {
+        indices(i) += component._1
+        values(i) += component._2
+        i += 1
+      }
+      SparseVector(indices, values)
+    }
+
+    /**
+      * Tells whether this instance contains components.
+      */
+    def isEmpty = components.isEmpty
+
+  }
+
+}
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/simwords/TextScrubber.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/simwords/TextScrubber.scala
new file mode 100644
index 0000000..4a71104
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/simwords/TextScrubber.scala
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.simwords
+
+/**
+  * Scrubs texts.
+  */
+class TextScrubber {
+
+  def splitAndScrub(line: String, collector: java.util.List[String]): Unit = {
+    for (token <- line.split("""\W""") if !token.isEmpty) {
+      collector.add(token.toLowerCase)
+    }
+  }
+
+}
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/simwords/Word2NVec.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/simwords/Word2NVec.scala
new file mode 100644
index 0000000..adbe05c
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/simwords/Word2NVec.scala
@@ -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 org.apache.wayang.apps.simwords
+
+import org.apache.wayang.apps.util.{ExperimentDescriptor, Parameters, ProfileDBHelper}
+import org.apache.wayang.api._
+import org.apache.wayang.apps.util.ProfileDBHelper
+import org.apache.wayang.commons.util.profiledb.model.Experiment
+import org.apache.wayang.core.api.{Configuration, WayangContext}
+import org.apache.wayang.core.function.ExecutionContext
+import org.apache.wayang.core.function.FunctionDescriptor.ExtendedSerializableFunction
+import org.apache.wayang.core.optimizer.ProbabilisticDoubleInterval
+import org.apache.wayang.core.optimizer.costs.LoadProfileEstimators
+import org.apache.wayang.core.plugin.Plugin
+import org.apache.wayang.core.util.fs.FileSystems
+
+/**
+  * This app maps words from a corpus to vectors describing that words average neighborhood. The vector components
+  * correspond to words and the values determine to some probability of finding that word in the given words neighborhood.
+  *
+  * <p>Note the UDF load properties `wayang.apps.simwords.udfs.create-neighborhood.load`.</p>
+  */
+class Word2NVec(plugins: Plugin*) {
+
+  def apply(inputFile: String,
+            minWordOccurrences: Int,
+            neighborhoodReach: Int,
+            wordsPerLine: ProbabilisticDoubleInterval,
+            outputFile: String)
+           (implicit experiment: Experiment,
+            configuration: Configuration) = {
+
+    // Initialize.
+    val wayangCtx = new WayangContext(configuration)
+    plugins.foreach(wayangCtx.register)
+    val planBuilder = new PlanBuilder(wayangCtx)
+      .withJobName(
+        jobName = s"Word2NVec ($inputFile, reach=$neighborhoodReach, output=$outputFile)"
+      ).withExperiment(experiment)
+      .withUdfJarsOf(this.getClass)
+
+    // Create the word dictionary
+    val _minWordOccurrences = minWordOccurrences
+    val wordIds = planBuilder
+      .readTextFile(inputFile).withName("Read corpus (1)")
+      .flatMapJava(new ScrubFunction, selectivity = wordsPerLine).withName("Split & scrub")
+      .map(word => (word, 1)).withName("Add word counter")
+      .reduceByKey(_._1, (wc1, wc2) => (wc1._1, wc1._2 + wc2._2)).withName("Sum word counters")
+      .withCardinalityEstimator((in: Long) => math.round(in * 0.01))
+      .filter(_._2 >= _minWordOccurrences, selectivity = 10d / (9d + minWordOccurrences))
+      .withName("Filter frequent words")
+      .map(_._1).withName("Strip word counter")
+      .zipWithId.withName("Zip with ID")
+      .map(t => (t.field1, t.field0.toInt)).withName("Convert ID attachment")
+
+
+    // Create the word neighborhood vectors.
+    val wordVectors = planBuilder
+      .readTextFile(inputFile).withName("Read corpus (2)")
+      .flatMapJava(
+        new CreateWordNeighborhoodFunction(neighborhoodReach, "wordIds"),
+        selectivity = wordsPerLine,
+        udfLoad = LoadProfileEstimators.createFromSpecification("wayang.apps.simwords.udfs.create-neighborhood.load", configuration)
+
+      )
+      .withBroadcast(wordIds, "wordIds")
+      .withName("Create word vectors")
+      .reduceByKey(_._1, (wv1, wv2) => (wv1._1, wv1._2 + wv2._2)).withName("Add word vectors")
+      .map { wv =>
+        wv._2.normalize(); wv
+      }.withName("Normalize word vectors")
+
+    // Enhance the word vectors by joining the actual word and write to an output file.
+    wordVectors
+      .mapJava(new ExtendWordVector)
+      .withBroadcast(wordIds, "wordIds")
+      .withName("Extend word vectors")
+      .writeTextFile(outputFile, wv => s"${wv._1};${wv._2};${wv._3.toDictionaryString}")
+  }
+
+}
+
+object Word2NVec extends ExperimentDescriptor {
+
+  override def version = "0.1.0"
+
+  def main(args: Array[String]): Unit = {
+    if (args.isEmpty) {
+      println(s"Usage: <main class> ${Parameters.experimentHelp} <plugin(,plugin)*> <input file> <min word occurrences> <neighborhood reach> <output file> [<words per line (from..to)>]")
+      sys.exit(1)
+    }
+
+    implicit val configuration = new Configuration
+    implicit val experiment = Parameters.createExperiment(args(0), this)
+    val plugins = Parameters.loadPlugins(args(1))
+    experiment.getSubject.addConfiguration("plugins", args(1))
+    val inputFile = args(2)
+    experiment.getSubject.addConfiguration("input", args(2))
+    val minWordOccurrences = args(3).toInt
+    experiment.getSubject.addConfiguration("minWordOccurrences", args(3))
+    val neighborhoodRead = args(4).toInt
+    experiment.getSubject.addConfiguration("neighborhoodReach", args(4))
+    val outputFile = args(5)
+    experiment.getSubject.addConfiguration("output", outputFile)
+    val wordsPerLine = if (args.length >= 7) {
+      experiment.getSubject.addConfiguration("wordsPerLine", args(6))
+      Parameters.parseAny(args(6)).asInstanceOf[ProbabilisticDoubleInterval]
+    } else new ProbabilisticDoubleInterval(100, 10000, 0.9)
+
+    val word2NVec = new Word2NVec(plugins: _*)
+    word2NVec(inputFile, minWordOccurrences, neighborhoodRead, wordsPerLine, outputFile)
+
+    // Store experiment data.
+    val inputFileSize = FileSystems.getFileSize(inputFile)
+    if (inputFileSize.isPresent) experiment.getSubject.addConfiguration("inputSize", inputFileSize.getAsLong)
+    ProfileDBHelper.store(experiment, configuration)
+
+  }
+}
+
+/**
+  * Extend `(word ID, neighborhood vector)` elements to `(word ID, word, neighborhood vector)` elements.
+  * <p>
+  * Accepts broadcast `wordIds` of `(word, word ID)` pairs.
+  */
+private[simwords] class ExtendWordVector
+  extends ExtendedSerializableFunction[(Int, SparseVector), (Int, String, SparseVector)] {
+
+  private var words: Map[Int, String] = _
+
+  /**
+    * Called before this instance is actually executed.
+    *
+    * @param ctx the { @link ExecutionContext}
+    */
+  override def open(ctx: ExecutionContext): Unit = {
+    import scala.collection.JavaConversions._
+    this.words = ctx.getBroadcast[(String, Int)]("wordIds").map(_.swap).toMap
+  }
+
+  override def apply(t: (Int, SparseVector)): (Int, String, SparseVector) = (t._1, this.words.getOrElse(t._1, "(unknown)"), t._2)
+}
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/sindy/Sindy.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/sindy/Sindy.scala
new file mode 100644
index 0000000..01db9ea
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/sindy/Sindy.scala
@@ -0,0 +1,217 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.sindy
+
+import java.lang.Iterable
+import java.util
+import org.apache.wayang.api._
+import org.apache.wayang.apps
+import Sindy.{CellCreator, CellMerger, IndCandidateGenerator, IndCandidateMerger}
+import org.apache.wayang.apps.util.{ExperimentDescriptor, Parameters, ProfileDBHelper, StdOut}
+import org.apache.wayang.apps.util.ProfileDBHelper
+import org.apache.wayang.commons.util.profiledb.model.Experiment
+import org.apache.wayang.core.api.{Configuration, WayangContext}
+import org.apache.wayang.core.function.FunctionDescriptor.{SerializableBinaryOperator, SerializableFunction}
+import org.apache.wayang.core.plugin.Plugin
+import org.apache.wayang.core.util.fs.FileSystems
+
+/**
+  * This is a Apache Wayang-based implementation of the SINDY algorithm.
+  */
+class Sindy(plugins: Plugin*) {
+
+  /**
+    * Execute the SINDY algorithm.
+    *
+    * @param paths         input file or directory URLs
+    * @param seperator     CSV separator in the files
+    * @param configuration Apache Wayang configuration
+    * @param experiment    the experiment to log measurements to
+    * @return the INDs
+    */
+  def apply(paths: Seq[String], seperator: Char = ';')
+           (implicit configuration: Configuration, experiment: Experiment) = {
+    val wayangContext = new WayangContext(configuration)
+    plugins.foreach(wayangContext.register)
+    val planBuilder = new PlanBuilder(wayangContext)
+      .withJobName(s"Sindy ($paths)")
+      .withExperiment(experiment)
+      .withUdfJarsOf(classOf[Sindy])
+
+    val fileColumnIdOffsets = paths.flatMap(resolveDirs).zipWithIndex.map { case (url, index) => (url, index * 1000) }
+    val allCells = fileColumnIdOffsets
+      .map { case (path, offset) =>
+        planBuilder
+          .readTextFile(path).withName(s"Load $path")
+          .flatMapJava(new CellCreator(offset, seperator)).withName(s"Create cells for $path")
+      }
+      .reduce(_ union _)
+
+    val rawInds = allCells
+      .map(cell => (cell._1, Array(cell._2))).withName("Prepare cell merging")
+      .reduceByKeyJava(toSerializableFunction(_._1), new CellMerger).withName("Merge cells")
+      .flatMapJava(new IndCandidateGenerator).withName("Generate IND candidate sets")
+      .reduceByKeyJava(toSerializableFunction(_._1), new IndCandidateMerger).withName("Merge IND candidate sets")
+      .filter(_._2.length > 0).withName("Filter empty candidate sets")
+      .collect()
+
+    def resolveColumnId(id: Int) = fileColumnIdOffsets
+      .find { case (file, offset) => offset <= id && offset + 1000 > id } match {
+      case Some((file, offset)) => s"$file[${id - offset}]"
+      case _ => s"???[$id]"
+    }
+
+    rawInds.map {
+      case (dep, refs) => (s"${resolveColumnId(dep)}", refs.map(resolveColumnId).toSeq)
+    }
+  }
+
+  /**
+    * If the given URL is a directory, list all its files recursively.
+    *
+    * @param url the URL
+    * @return the file URLs
+    */
+  def resolveDirs(url: String): scala.Iterable[String] = {
+    import scala.collection.JavaConversions._
+    val fs = FileSystems.requireFileSystem(url)
+    if (fs.isDirectory(url)) fs.listChildren(url).flatMap(resolveDirs) else Seq(url)
+  }
+
+}
+
+/**
+  * Companion object for [[Sindy]].
+  */
+object Sindy extends ExperimentDescriptor {
+
+  def version = "0.1.0"
+
+  def main(args: Array[String]): Unit = {
+    // Parse parameters.
+    if (args.isEmpty) {
+      sys.error(s"Usage: <main class> ${Parameters.experimentHelp} <plugin>(,<plugin>)* <CSV separator> <input URL>(;<input URL>)*")
+      sys.exit(1)
+    }
+
+    implicit val configuration = new Configuration
+    implicit val experiment = Parameters.createExperiment(args(0), this)
+    val plugins = Parameters.loadPlugins(args(1))
+    experiment.getSubject.addConfiguration("plugins", args(1))
+    val separator = if (args(2).length == 1) args(2).charAt(0) else args(2) match {
+      case "tab" => '\t'
+      case "\\t" => '\t'
+      case "comma" => ','
+      case "semicolon" => ';'
+      case "\\|" => '|'
+      case "pipe" => '|'
+      case other: String => throw new IllegalArgumentException("Unknown separator.")
+    }
+    val inputUrls = args(3).split(";")
+    experiment.getSubject.addConfiguration("inputs", inputUrls)
+
+    // Prepare the PageRank.
+    val sindy = new Sindy(plugins: _*)
+
+    // Run the PageRank.
+    val inds = sindy(inputUrls.toSeq, separator).toSeq
+
+    // Store experiment data.
+    val inputFileSizes = inputUrls.map(url => FileSystems.getFileSize(url))
+    if (inputFileSizes.forall(_.isPresent))
+      experiment.getSubject.addConfiguration("inputSize", inputFileSizes.map(_.getAsLong).sum)
+    ProfileDBHelper.store(experiment, configuration)
+
+    // Print the result.
+    StdOut.printLimited(inds)
+  }
+
+  /**
+    * UDF to parse a CSV row and create cells.
+    *
+    * @param offset the column ID offset for the input CSV rows
+    */
+  class CellCreator(val offset: Int, val separator: Char) extends SerializableFunction[String, java.lang.Iterable[(String, Int)]] {
+
+    override def apply(row: String): Iterable[(String, Int)] = {
+      val fields = row.split(separator)
+      val cells = new util.ArrayList[(String, Int)](fields.length)
+      var columnId = offset
+      for (field <- fields) {
+        cells.add((field, columnId))
+        columnId += 1
+      }
+      cells
+    }
+  }
+
+  /**
+    * UDF to merge the column IDs of two cells.
+    */
+  class CellMerger extends SerializableBinaryOperator[(String, Array[Int])] {
+
+    import scala.collection.mutable
+
+    lazy val merger = mutable.Set[Int]()
+
+    override def apply(cell1: (String, Array[Int]), cell2: (String, Array[Int])): (String, Array[Int]) = {
+      merger.clear()
+      for (columnId <- cell1._2) merger += columnId
+      for (columnId <- cell2._2) merger += columnId
+      (cell1._1, merger.toArray)
+    }
+  }
+
+  /**
+    * UDF to create IND candidates from a cell group.
+    */
+  class IndCandidateGenerator extends SerializableFunction[(String, Array[Int]), java.lang.Iterable[(Int, Array[Int])]] {
+
+    override def apply(cellGroup: (String, Array[Int])): java.lang.Iterable[(Int, Array[Int])] = {
+      val columnIds = cellGroup._2
+      val result = new util.ArrayList[(Int, Array[Int])](columnIds.length)
+      for (i <- columnIds.indices) {
+        val refColumnIds = new Array[Int](columnIds.length - 1)
+        java.lang.System.arraycopy(columnIds, 0, refColumnIds, 0, i)
+        java.lang.System.arraycopy(columnIds, i + 1, refColumnIds, i, refColumnIds.length - i)
+        result.add((columnIds(i), refColumnIds))
+      }
+      result
+    }
+  }
+
+  /**
+    * UDF to merge two IND candidates.
+    */
+  class IndCandidateMerger extends SerializableBinaryOperator[(Int, Array[Int])] {
+
+    import scala.collection.mutable
+
+    lazy val merger = mutable.Set[Int]()
+
+    override def apply(indc1: (Int, Array[Int]), indc2: (Int, Array[Int])): (Int, Array[Int]) = {
+      merger.clear()
+      for (columnId <- indc1._2) merger += columnId
+      (indc1._1, indc2._2.filter(merger.contains))
+    }
+
+  }
+
+
+}
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/tpch/CsvUtils.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/tpch/CsvUtils.scala
new file mode 100644
index 0000000..6f4c0fb
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/tpch/CsvUtils.scala
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.tpch
+
+import org.apache.wayang.core.api.exception.WayangException
+
+/**
+  * Utilities for handling CSV files.
+  */
+object CsvUtils {
+
+  val dateRegex = """(\d{4})-(\d{2})-(\d{2})""".r
+
+  /**
+    * Parse a date string (see [[CsvUtils.dateRegex]]) into a custom [[Int]] representation.
+    *
+    * @param str to be parsed
+    * @return the [[Int]] representation
+    */
+  def parseDate(str: String): Int = str match {
+    case CsvUtils.dateRegex(year, month, day) => year.toInt * 365 + month.toInt * 30 + day.toInt
+    case other: String => throw new WayangException(s"Cannot parse '$other' as date.")
+  }
+
+}
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/tpch/SqlUtils.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/tpch/SqlUtils.scala
new file mode 100644
index 0000000..2916738
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/tpch/SqlUtils.scala
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.tpch
+
+/**
+  * Utilities to handle SQL-related applications.
+  */
+object SqlUtils {
+
+  /**
+    * Convert a [[java.sql.Date]] to our internal [[Int]] representation.
+    * @param date to be converted
+    * @return the [[Int]] representation
+    */
+  @SuppressWarnings(Array("deprecated"))
+  def toInt(date: java.sql.Date) = (date.getYear + 1900) * 10000 + date.getMonth * 100 + date.getDate
+
+}
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/tpch/TpcH.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/tpch/TpcH.scala
new file mode 100644
index 0000000..60e35a7
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/tpch/TpcH.scala
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.tpch
+
+import org.apache.wayang.apps.tpch.queries.{Query1, Query3Database, Query3File, Query3Hybrid}
+import org.apache.wayang.apps.util.{Parameters, ProfileDBHelper, StdOut}
+import org.apache.wayang.apps.tpch.queries.Query1
+import org.apache.wayang.apps.util.ProfileDBHelper
+import org.apache.wayang.commons.util.profiledb.model.Experiment
+import org.apache.wayang.core.api.Configuration
+import org.apache.wayang.jdbc.platform.JdbcPlatformTemplate
+import org.apache.wayang.postgres.Postgres
+import org.apache.wayang.postgres.operators.PostgresTableSource
+import org.apache.wayang.sqlite3.Sqlite3
+import org.apache.wayang.sqlite3.operators.Sqlite3TableSource
+
+import scala.collection.JavaConversions._
+
+/**
+  * This app adapts some TPC-H queries.
+  */
+object TpcH {
+
+  def main(args: Array[String]): Unit = {
+    if (args.isEmpty) {
+      println(s"Usage: <main class> ${Parameters.experimentHelp} <plugin(,plugin)*> <TPC-H config URL> <query> [<query args>*]")
+      sys.exit(1)
+    }
+
+    val experimentArg = args(0)
+    val plugins = Parameters.loadPlugins(args(1))
+    val configUrl = args(2)
+    val queryName = args(3)
+
+    val jdbcPlatform = {
+      val jdbcPlatforms = plugins
+        .flatMap(_.getRequiredPlatforms)
+        .filter(_.isInstanceOf[JdbcPlatformTemplate])
+        .distinct
+      if (jdbcPlatforms.size == 1) jdbcPlatforms.head.asInstanceOf[JdbcPlatformTemplate]
+      else if (jdbcPlatforms.isEmpty) null
+      else throw new IllegalArgumentException(s"Detected multiple databases: ${jdbcPlatforms.mkString(", ")}.")
+    }
+
+    val createTableSource =
+      if (jdbcPlatform == null) {
+        (table: String, columns: Seq[String]) => throw new IllegalStateException("No database plugin detected.")
+      } else if (jdbcPlatform.equals(Sqlite3.platform)) {
+        (table: String, columns: Seq[String]) => new Sqlite3TableSource(table, columns: _*)
+      } else if (jdbcPlatform.equals(Postgres.platform)) {
+        (table: String, columns: Seq[String]) => new PostgresTableSource(table, columns: _*)
+      } else {
+        throw new IllegalArgumentException(s"Unsupported database: $jdbcPlatform.")
+      }
+
+    val configuration = new Configuration
+    configuration.load(configUrl)
+
+    var experiment: Experiment = null
+    queryName match {
+      case "Q1" =>
+        val query = new Query1(plugins: _*)
+        experiment = Parameters.createExperiment(experimentArg, query)
+        experiment.getSubject.addConfiguration("plugins", args(1))
+        experiment.getSubject.addConfiguration("query", args(3))
+        val result = query(configuration, jdbcPlatform, createTableSource)(experiment)
+        StdOut.printLimited(result, 10)
+      case "Q3File" =>
+        val query = new Query3File(plugins: _*)
+        experiment = Parameters.createExperiment(experimentArg, query)
+        experiment.getSubject.addConfiguration("plugins", args(1))
+        experiment.getSubject.addConfiguration("query", args(3))
+        val result = query(configuration)(experiment)
+        StdOut.printLimited(result, 10)
+      case "Q3" =>
+        val query = new Query3Database(plugins: _*)
+        experiment = Parameters.createExperiment(experimentArg, query)
+        experiment.getSubject.addConfiguration("plugins", args(1))
+        experiment.getSubject.addConfiguration("query", args(3))
+        val result = query(configuration, jdbcPlatform, createTableSource)(experiment)
+        StdOut.printLimited(result, 10)
+      case "Q3Hybrid" =>
+        val query = new Query3Hybrid(plugins: _*)
+        experiment = Parameters.createExperiment(experimentArg, query)
+        experiment.getSubject.addConfiguration("plugins", args(1))
+        experiment.getSubject.addConfiguration("query", args(3))
+        val result = query(configuration, jdbcPlatform, createTableSource)(experiment)
+        StdOut.printLimited(result, 10)
+      case other: String => {
+        println(s"Unknown query: $other")
+        sys.exit(1)
+      }
+    }
+
+    // Store experiment data.
+    ProfileDBHelper.store(experiment, configuration)
+  }
+
+}
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/tpch/data/Customer.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/tpch/data/Customer.scala
new file mode 100644
index 0000000..d5b79b3
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/tpch/data/Customer.scala
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.tpch.data
+
+/**
+  * Represents elements from the TPC-H `CUSTOMER` table.
+  */
+case class Customer(custKey: Long,
+                    name: String,
+                    address: String,
+                    nationKey: Long,
+                    phone: String,
+                    acctbal: Double,
+                    mktSegment: String,
+                    comment: String)
+
+object Customer {
+
+  val fields = IndexedSeq("c_custkey", "c_name", "c_address", "c_nationkey", "c_phone", "c_acctbal", "c_mktsegment", "c_comment")
+
+  /**
+    * Parse a CSV row into a [[Customer]] instance.
+    *
+    * @param csv the [[String]] to parse
+    * @return the [[Customer]]
+    */
+  def parseCsv(csv: String): Customer = {
+    val fields = csv.split("\\|")
+
+    Customer(
+      fields(0).toLong,
+      fields(1),
+      fields(2),
+      fields(3).toLong,
+      fields(4),
+      fields(5).toDouble,
+      fields(6).trim,
+      fields(7)
+    )
+  }
+
+}
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/tpch/data/LineItem.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/tpch/data/LineItem.scala
new file mode 100644
index 0000000..dbdf706
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/tpch/data/LineItem.scala
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.tpch.data
+
+import org.apache.wayang.apps.tpch.CsvUtils
+import org.apache.wayang.core.api.exception.WayangException
+
+/**
+  * Represents elements from the TPC-H `LINEITEM` table.
+  */
+case class LineItem(orderKey: Long,
+                    partKey: Long,
+                    suppKey: Long,
+                    lineNumber: Int,
+                    quantity: Double,
+                    extendedPrice: Double,
+                    discount: Double,
+                    tax: Double,
+                    returnFlag: Char,
+                    lineStatus: Char,
+                    shipDate: Int,
+                    commitDate: Int,
+                    receiptDate: Int,
+                    shipInstruct: String,
+                    shipMode: String,
+                    comment: String)
+
+object LineItem {
+
+  val fields = IndexedSeq("l_orderkey",
+    "l_partkey",
+    "l_suppkey",
+    "l_linenumber",
+    "l_quantity",
+    "l_extendedprice",
+    "l_discount",
+    "l_tax",
+    "l_returnflag",
+    "l_linestatus",
+    "l_shipdate",
+    "l_commitdate",
+    "l_receiptdate",
+    "l_shipinstruct",
+    "l_shipmode",
+    "l_comment")
+
+  /**
+    * Parse a CSV row into a [[LineItem]] instance.
+    *
+    * @param csv the [[String]] to parse
+    * @return the [[LineItem]]
+    */
+  def parseCsv(csv: String): LineItem = {
+    val fields = csv.split("\\|")
+
+    try {
+      LineItem(
+        fields(0).toLong,
+        fields(1).toLong,
+        fields(2).toLong,
+        fields(3).toInt,
+        fields(4).toDouble,
+        fields(5).toDouble,
+        fields(6).toDouble,
+        fields(7).toDouble,
+        fields(8).charAt(0),
+        fields(9).charAt(0),
+        CsvUtils.parseDate(fields(10)),
+        CsvUtils.parseDate(fields(11)),
+        CsvUtils.parseDate(fields(12)),
+        fields(13),
+        fields(14),
+        fields(15)
+      )
+    } catch {
+      case e: Exception => throw new WayangException(s"Could not parse '$csv' (${fields.mkString(", ")}.", e)
+    }
+  }
+
+}
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/tpch/data/Order.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/tpch/data/Order.scala
new file mode 100644
index 0000000..acbc187
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/tpch/data/Order.scala
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.tpch.data
+
+import org.apache.wayang.apps.tpch.CsvUtils
+
+/**
+  * Represents elements from the TPC-H `ORDERS` table.
+  */
+case class Order(orderKey: Long,
+                 custKey: Long,
+                 orderStatus: Char,
+                 totalPrice: Double,
+                 orderDate: Int,
+                 orderPriority: String,
+                 clerk: String,
+                 shipPrioritiy: Int,
+                 comment: String)
+
+object Order {
+
+  val fields = IndexedSeq(
+    "o_orderkey",
+    "o_custkey",
+    "o_orderstatus",
+    "o_totalprice",
+    "o_orderdate",
+    "o_orderpriority",
+    "o_clerk",
+    "o_shippriority",
+    "o_comment"
+  )
+
+  /**
+    * Parse a CSV row into a [[Order]] instance.
+    *
+    * @param csv the [[String]] to parse
+    * @return the [[Order]]
+    */
+  def parseCsv(csv: String): Order = {
+    val fields = csv.split("\\|")
+
+    Order(
+      fields(0).toLong,
+      fields(1).toLong,
+      fields(2).charAt(0),
+      fields(3).toDouble,
+      CsvUtils.parseDate(fields(4)),
+      fields(5),
+      fields(6),
+      fields(7).toInt,
+      fields(8)
+    )
+  }
+
+}
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/tpch/queries/Query1.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/tpch/queries/Query1.scala
new file mode 100644
index 0000000..01c6a3d
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/tpch/queries/Query1.scala
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.tpch.queries
+
+import org.apache.wayang.apps.tpch.CsvUtils
+import org.apache.wayang.apps.tpch.data.LineItem
+import org.apache.wayang.apps.util.ExperimentDescriptor
+import org.apache.wayang.api._
+import org.apache.wayang.commons.util.profiledb.model.Experiment
+import org.apache.wayang.core.api.{Configuration, WayangContext}
+import org.apache.wayang.core.plugin.Plugin
+import org.apache.wayang.jdbc.operators.JdbcTableSource
+import org.apache.wayang.jdbc.platform.JdbcPlatformTemplate
+
+/**
+  * Apache Wayang implementation of TPC-H Query 1.
+  *
+  * {{{
+  * select
+  *   l_returnflag,
+  *   l_linestatus,
+  *   sum(l_quantity) as sum_qty,
+  *   sum(l_extendedprice) as sum_base_price,
+  *   sum(l_extendedprice*(1-l_discount)) as sum_disc_price,
+  *   sum(l_extendedprice*(1-l_discount)*(1+l_tax)) as sum_charge,
+  *   avg(l_quantity) as avg_qty,
+  *   avg(l_extendedprice) as avg_price,
+  *   avg(l_discount) as avg_disc,
+  *   count(*) as count_order
+  * from
+  *   lineitem
+  * where
+  *   l_shipdate <= date '1998-12-01' - interval '[DELTA]' day (3)
+  * group by
+  *   l_returnflag,
+  *   l_linestatus
+  * order by
+  *   l_returnflag,
+  *   l_linestatus;
+  * }}}
+  */
+class Query1(plugins: Plugin*) extends ExperimentDescriptor {
+
+  override def version = "0.1.0"
+
+  def apply(configuration: Configuration,
+            jdbcPlatform: JdbcPlatformTemplate,
+            createTableSource: (String, Seq[String]) => JdbcTableSource,
+            delta: Int = 90)
+           (implicit experiment: Experiment): Iterable[Query1.Result] = {
+
+    val wayangCtx = new WayangContext(configuration)
+    plugins.foreach(wayangCtx.register)
+    val planBuilder = new PlanBuilder(wayangCtx)
+      .withJobName(s"TPC-H (${this.getClass.getSimpleName})")
+      .withUdfJarsOf(classOf[Query1])
+      .withExperiment(experiment)
+
+    experiment.getSubject.addConfiguration("jdbcUrl", configuration.getStringProperty(jdbcPlatform.jdbcUrlProperty))
+    experiment.getSubject.addConfiguration("delta", delta)
+
+    // Read, filter, and project the customer data.
+    val _delta = delta
+    val result = planBuilder
+      .readTable(createTableSource("LINEITEM", LineItem.fields))
+      .withName("Load LINEITEM table")
+
+      .filter(t => CsvUtils.parseDate(t.getString(10)) <= CsvUtils.parseDate("1998-12-01") - _delta,
+        sqlUdf = s"date(l_shipdate) <= date('1998-12-01', '- ${_delta} day')", selectivity = .25)
+      .withName("Filter line items")
+
+      .projectRecords(Seq("l_returnflag", "l_linestatus", "l_quantity", "l_extendedprice", "l_discount", "l_tax"))
+      .withName("Project line items")
+
+      .map(record => Query1.Result(
+        record.getString(0),
+        record.getString(1),
+        record.getDouble(2),
+        record.getDouble(3),
+        record.getDouble(3) * (1 - record.getDouble(4)),
+        record.getDouble(3) * (1 - record.getDouble(4)) * (1 + record.getDouble(5)),
+        record.getDouble(2),
+        record.getDouble(3),
+        record.getDouble(4),
+        1
+      ))
+      .withName("Calculate result fields")
+
+      .reduceByKey(
+        result => (result.l_returnflag, result.l_linestatus),
+        (r1, r2) => Query1.Result(
+          r1.l_returnflag,
+          r1.l_linestatus,
+          r1.sum_qty + r2.sum_qty,
+          r1.sum_base_price + r2.sum_base_price,
+          r1.sum_disc_price + r2.sum_disc_price,
+          r1.sum_charge + r2.sum_charge,
+          r1.avg_qty + r2.avg_qty,
+          r1.avg_price + r2.avg_price,
+          r1.avg_disc + r2.avg_disc,
+          r1.count_order + r2.count_order
+        )
+      )
+      .withName("Aggregate line items")
+
+      .map(result => Query1.Result(
+        result.l_returnflag,
+        result.l_linestatus,
+        result.sum_qty,
+        result.sum_base_price,
+        result.sum_disc_price,
+        result.sum_charge,
+        result.avg_qty / result.count_order,
+        result.avg_price / result.count_order,
+        result.avg_disc / result.count_order,
+        result.count_order
+      ))
+      .withName("Post-process line item aggregates")
+      .collect()
+
+    result
+  }
+
+}
+
+object Query1 {
+
+  case class Result(l_returnflag: String,
+                    l_linestatus: String,
+                    sum_qty: Double,
+                    sum_base_price: Double,
+                    sum_disc_price: Double,
+                    sum_charge: Double,
+                    avg_qty: Double,
+                    avg_price: Double,
+                    avg_disc: Double,
+                    count_order: Int)
+
+}
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/tpch/queries/Query3Database.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/tpch/queries/Query3Database.scala
new file mode 100644
index 0000000..d3ea4e7
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/tpch/queries/Query3Database.scala
@@ -0,0 +1,170 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.tpch.queries
+
+import org.apache.wayang.apps.tpch.CsvUtils
+import org.apache.wayang.apps.tpch.data.{Customer, LineItem, Order}
+import org.apache.wayang.apps.util.ExperimentDescriptor
+import org.apache.wayang.api._
+import org.apache.wayang.apps.tpch.data.LineItem
+import org.apache.wayang.commons.util.profiledb.model.Experiment
+import org.apache.wayang.core.api.{Configuration, WayangContext}
+import org.apache.wayang.core.plugin.Plugin
+import org.apache.wayang.jdbc.operators.JdbcTableSource
+import org.apache.wayang.jdbc.platform.JdbcPlatformTemplate
+
+/**
+  * Apache Wayang implementation of TPC-H Query 3.
+  *
+  * {{{
+  * select
+  *   l_orderkey,
+  *   sum(l_extendedprice*(1-l_discount)) as revenue,
+  *   o_orderdate,
+  *   o_shippriority
+  * from
+  *   customer,
+  *   orders,
+  *   lineitem
+  * where
+  *   c_mktsegment = '[SEGMENT]'
+  *   and c_custkey = o_custkey
+  *   and l_orderkey = o_orderkey
+  *   and o_orderdate < date '[DATE]'
+  *   and l_shipdate > date '[DATE]'
+  * group by
+  *   l_orderkey,
+  *   o_orderdate,
+  *   o_shippriority
+  * order by
+  *   revenue desc,
+  *   o_orderdate;
+  * }}}
+  */
+class Query3Database(plugins: Plugin*) extends ExperimentDescriptor {
+
+  override def version = "0.1.0"
+
+  def apply(configuration: Configuration,
+            jdbcPlatform: JdbcPlatformTemplate,
+            createTableSource: (String, Seq[String]) => JdbcTableSource,
+            segment: String = "BUILDING",
+            date: String = "1995-03-15")
+           (implicit experiment: Experiment) = {
+
+    val wayangCtx = new WayangContext(configuration)
+    plugins.foreach(wayangCtx.register)
+    val planBuilder = new PlanBuilder(wayangCtx)
+      .withJobName(s"TPC-H (${this.getClass.getSimpleName})")
+      .withUdfJarsOf(classOf[Query3Database])
+      .withExperiment(experiment)
+
+    val schema = configuration.getOptionalStringProperty("wayang.apps.tpch.schema").orElse(null)
+    def withSchema(table: String) = schema match {
+      case null => table
+      case str: String => s"$str.$table"
+    }
+
+    experiment.getSubject.addConfiguration("jdbcUrl", configuration.getStringProperty(jdbcPlatform.jdbcUrlProperty))
+    if (schema != null) experiment.getSubject.addConfiguration("schema", schema)
+    experiment.getSubject.addConfiguration("segment", segment)
+    experiment.getSubject.addConfiguration("date", date)
+
+    // Read, filter, and project the customer data.
+    val _segment = segment
+    val customerKeys = planBuilder
+      .readTable(createTableSource(withSchema("CUSTOMER"), Customer.fields))
+      .withName("Load CUSTOMER table")
+
+      .filter(_.getString(6) == _segment, sqlUdf = s"c_mktsegment LIKE '$segment%'", selectivity = .25)
+      .withName("Filter customers")
+
+      .projectRecords(Seq("c_custkey"))
+      .withName("Project customers")
+
+      .map(_.getLong(0))
+      .withName("Extract customer ID")
+
+    // Read, filter, and project the order data.
+    val _date = CsvUtils.parseDate(date)
+    val orders = planBuilder
+      .load(createTableSource(withSchema("ORDERS"), Order.fields))
+      .withName("Load ORDERS table")
+
+      .filter(t => CsvUtils.parseDate(t.getString(4)) > _date, sqlUdf = s"o_orderdate < date('$date')")
+      .withName("Filter orders")
+
+      .projectRecords(Seq("o_orderkey", "o_custkey", "o_orderdate", "o_shippriority"))
+      .withName("Project orders")
+
+      .map(order => (order.getLong(0), // orderKey
+        order.getLong(1), // custKey
+        CsvUtils.parseDate(order.getString(2)), // orderDate
+        order.getInt(3)) // shipPriority
+      )
+      .withName("Unpack orders")
+
+    // Read, filter, and project the line item data.
+    val lineItems = planBuilder
+      .readTable(createTableSource(withSchema("LINEITEM"), LineItem.fields))
+      .withName("Load LINEITEM table")
+
+      .filter(t => CsvUtils.parseDate(t.getString(10)) > _date, sqlUdf = s"l_shipDate > date('$date')")
+      .withName("Filter line items")
+
+      .projectRecords(Seq("l_orderkey", "l_extendedprice", "l_discount"))
+      .withName("Project line items")
+
+      .map(li => (
+        li.getLong(0), //li.orderKey,
+        li.getDouble(1) * (1 - li.getDouble(2)) //li.extendedPrice * (1 - li.discount)
+      ))
+      .withName("Extract line item data")
+
+    // Join and aggregate the different datasets.
+    customerKeys
+      .join[(Long, Long, Int, Int), Long](identity, orders, _._2)
+      .withName("Join customers with orders")
+
+      .map(_.field1) // (orderKey, custKey, orderDate, shipPriority)
+      .withName("Project customer-order join product")
+
+      .join[(Long, Double), Long](_._1, lineItems, _._1)
+      .withName("Join CO with line items")
+
+      .map(coli => Query3Result(
+        orderKey = coli.field1._1,
+        revenue = coli.field1._2,
+        orderDate = coli.field0._3,
+        shipPriority = coli.field0._4
+      ))
+      .withName("Project CO-line-item join product")
+
+      .reduceByKey(
+        t => (t.orderKey, t.orderDate, t.shipPriority),
+        (t1, t2) => {
+          t1.revenue += t2.revenue;
+          t2
+        }
+      )
+      .withName("Aggregate revenue")
+      .collect()
+  }
+
+}
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/tpch/queries/Query3File.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/tpch/queries/Query3File.scala
new file mode 100644
index 0000000..171e184
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/tpch/queries/Query3File.scala
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.tpch.queries
+
+import org.apache.wayang.apps.tpch.CsvUtils
+import org.apache.wayang.apps.tpch.data.{Customer, LineItem, Order}
+import org.apache.wayang.apps.util.ExperimentDescriptor
+import org.apache.wayang.api._
+import org.apache.wayang.apps.tpch.data.LineItem
+import org.apache.wayang.commons.util.profiledb.model.Experiment
+import org.apache.wayang.core.api.{Configuration, WayangContext}
+import org.apache.wayang.core.plugin.Plugin
+
+/**
+  * Apache Wayang implementation of TPC-H Query 3.
+  *
+  * {{{
+  * select
+  *   l_orderkey,
+  *   sum(l_extendedprice*(1-l_discount)) as revenue,
+  *   o_orderdate,
+  *   o_shippriority
+  * from
+  *   customer,
+  *   orders,
+  *   lineitem
+  * where
+  *   c_mktsegment = '[SEGMENT]'
+  *   and c_custkey = o_custkey
+  *   and l_orderkey = o_orderkey
+  *   and o_orderdate < date '[DATE]'
+  *   and l_shipdate > date '[DATE]'
+  * group by
+  *   l_orderkey,
+  *   o_orderdate,
+  *   o_shippriority
+  * order by
+  *   revenue desc,
+  *   o_orderdate;
+  * }}}
+  */
+class Query3File(plugins: Plugin*) extends ExperimentDescriptor {
+
+  override def version = "0.1.0"
+
+  def apply(configuration: Configuration,
+            segment: String = "BUILDING",
+            date: String = "1995-03-15")
+           (implicit experiment: Experiment) = {
+
+    val wayangCtx = new WayangContext(configuration)
+    plugins.foreach(wayangCtx.register)
+    val planBuilder = new PlanBuilder(wayangCtx)
+      .withUdfJarsOf(classOf[Query3Database])
+      .withExperiment(experiment)
+      .withJobName(s"TPC-H (${this.getClass.getSimpleName})")
+
+    val customerFile = configuration.getStringProperty("wayang.apps.tpch.csv.customer")
+    val ordersFile = configuration.getStringProperty("wayang.apps.tpch.csv.orders")
+    val lineitemFile = configuration.getStringProperty("wayang.apps.tpch.csv.lineitem")
+
+    experiment.getSubject.addConfiguration("customerInput", customerFile)
+    experiment.getSubject.addConfiguration("ordersInput", ordersFile)
+    experiment.getSubject.addConfiguration("lineitemInput", lineitemFile)
+    experiment.getSubject.addConfiguration("segment", segment)
+    experiment.getSubject.addConfiguration("date", date)
+
+    // Read, filter, and project the customer data.
+    val _segment = segment
+    val customerKeys = planBuilder
+      .readTextFile(customerFile)
+      .withName("Read customers")
+      .map(Customer.parseCsv)
+      .withName("Parse customers")
+
+      .filter(_.mktSegment == _segment, selectivity = .25)
+      .withName("Filter customers")
+
+      .map(_.custKey)
+      .withName("Project customers")
+
+    // Read, filter, and project the order data.
+    val _date = CsvUtils.parseDate(date)
+    val orders = planBuilder
+      .readTextFile(ordersFile)
+      .withName("Read orders")
+      .map(Order.parseCsv)
+      .withName("Parse orders")
+
+      .filter(_.orderDate < _date)
+      .withName("Filter orders")
+
+      .map(order => (order.orderKey, order.custKey, order.orderDate, order.shipPrioritiy))
+      .withName("Project orders")
+
+    // Read, filter, and project the line item data.
+    val lineItems = planBuilder
+      .readTextFile(lineitemFile)
+      .withName("Read line items")
+      .map(LineItem.parseCsv)
+      .withName("Parse line items")
+
+      .filter(_.shipDate > _date)
+      .withName("Filter line items")
+
+      .map(li => (li.orderKey, li.extendedPrice * (1 - li.discount)))
+      .withName("Project line items")
+
+    // Join and aggregate the different datasets.
+    customerKeys
+      .join[(Long, Long, Int, Int), Long](identity, orders, _._2)
+      .withName("Join customers with orders")
+      .map(_.field1) // (orderKey, custKey, orderDate, shipPriority)
+      .withName("Project customer-order join product")
+
+      .join[(Long, Double), Long](_._1, lineItems, _._1)
+      .withName("Join CO with line items")
+      .map(coli => Query3Result(
+        orderKey = coli.field1._1,
+        revenue = coli.field1._2,
+        orderDate = coli.field0._3,
+        shipPriority = coli.field0._4
+      ))
+      .withName("Project CO-line-item join product")
+
+      .reduceByKey(
+        t => (t.orderKey, t.orderDate, t.shipPriority),
+        (t1, t2) => {
+          t1.revenue += t2.revenue;
+          t2
+        }
+      )
+      .withName("Aggregate revenue")
+      .collect()
+  }
+
+}
+
+case class Query3Result(orderKey: Long, var revenue: Double, orderDate: Int, shipPriority: Int)
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/tpch/queries/Query3Hybrid.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/tpch/queries/Query3Hybrid.scala
new file mode 100644
index 0000000..06a8cc7
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/tpch/queries/Query3Hybrid.scala
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.tpch.queries
+
+import org.apache.wayang.apps.tpch.CsvUtils
+import org.apache.wayang.apps.tpch.data.{Customer, LineItem, Order}
+import org.apache.wayang.apps.util.ExperimentDescriptor
+import org.apache.wayang.api._
+import org.apache.wayang.apps.tpch.data.LineItem
+import org.apache.wayang.commons.util.profiledb.model.Experiment
+import org.apache.wayang.core.api.{Configuration, WayangContext}
+import org.apache.wayang.core.plugin.Plugin
+import org.apache.wayang.jdbc.operators.JdbcTableSource
+import org.apache.wayang.jdbc.platform.JdbcPlatformTemplate
+
+/**
+  * Apache Wayang implementation of TPC-H Query 3.
+  *
+  * {{{
+  * select
+  *   l_orderkey,
+  *   sum(l_extendedprice*(1-l_discount)) as revenue,
+  *   o_orderdate,
+  *   o_shippriority
+  * from
+  *   customer,
+  *   orders,
+  *   lineitem
+  * where
+  *   c_mktsegment = '[SEGMENT]'
+  *   and c_custkey = o_custkey
+  *   and l_orderkey = o_orderkey
+  *   and o_orderdate < date '[DATE]'
+  *   and l_shipdate > date '[DATE]'
+  * group by
+  *   l_orderkey,
+  *   o_orderdate,
+  *   o_shippriority
+  * order by
+  *   revenue desc,
+  *   o_orderdate;
+  * }}}
+  */
+class Query3Hybrid(plugins: Plugin*) extends ExperimentDescriptor {
+
+  override def version = "0.1.0"
+
+  def apply(configuration: Configuration,
+            jdbcPlatform: JdbcPlatformTemplate,
+            createTableSource: (String, Seq[String]) => JdbcTableSource,
+            segment: String = "BUILDING",
+            date: String = "1995-03-15")
+           (implicit experiment: Experiment) = {
+
+    val wayangCtx = new WayangContext(configuration)
+    plugins.foreach(wayangCtx.register)
+    val planBuilder = new PlanBuilder(wayangCtx)
+      .withJobName(s"TPC-H (${this.getClass.getSimpleName})")
+      .withUdfJarsOf(classOf[Query3Hybrid])
+      .withExperiment(experiment)
+
+    val schema = configuration.getOptionalStringProperty("wayang.apps.tpch.schema").orElse(null)
+    def withSchema(table: String) = schema match {
+      case null => table
+      case str: String => s"$str.$table"
+    }
+    val lineitemFile = configuration.getStringProperty("wayang.apps.tpch.csv.lineitem")
+
+    experiment.getSubject.addConfiguration("jdbcUrl", configuration.getStringProperty(jdbcPlatform.jdbcUrlProperty))
+    if (schema != null) experiment.getSubject.addConfiguration("schema", schema)
+    experiment.getSubject.addConfiguration("lineitemInput", lineitemFile)
+    experiment.getSubject.addConfiguration("segment", segment)
+    experiment.getSubject.addConfiguration("date", date)
+
+    // Read, filter, and project the customer data.
+    val _segment = segment
+    val customerKeys = planBuilder
+      .readTable(createTableSource(withSchema("CUSTOMER"), Customer.fields))
+      .withName("Load CUSTOMER table")
+
+      .filter(_.getString(6) == _segment, sqlUdf = s"c_mktsegment LIKE '$segment%'", selectivity = .25)
+      .withName("Filter customers")
+
+      .projectRecords(Seq("c_custkey"))
+      .withName("Project customers")
+
+      .map(_.getLong(0))
+      .withName("Extract customer ID")
+
+    // Read, filter, and project the order data.
+    val _date = CsvUtils.parseDate(date)
+    val orders = planBuilder
+      .load(createTableSource(withSchema("ORDERS"), Order.fields))
+      .withName("Load ORDERS table")
+
+      .filter(t => CsvUtils.parseDate(t.getString(4)) > _date, sqlUdf = s"o_orderdate < date('$date')")
+      .withName("Filter orders")
+
+      .projectRecords(Seq("o_orderkey", "o_custkey", "o_orderdate", "o_shippriority"))
+      .withName("Project orders")
+
+      .map(order => (order.getLong(0), // orderKey
+        order.getLong(1), // custKey
+        CsvUtils.parseDate(order.getString(2)), // orderDate
+        order.getInt(3)) // shipPriority
+      )
+      .withName("Unpack orders")
+
+    // Read, filter, and project the line item data.
+    val lineItems = planBuilder
+      .readTextFile(lineitemFile)
+      .withName("Read line items")
+      .map(LineItem.parseCsv)
+      .withName("Parse line items")
+
+      .filter(_.shipDate > _date)
+      .withName("Filter line items")
+
+      .map(li => (li.orderKey, li.extendedPrice * (1 - li.discount)))
+      .withName("Project line items")
+
+    // Join and aggregate the different datasets.
+    customerKeys
+      .join[(Long, Long, Int, Int), Long](identity, orders, _._2)
+      .withName("Join customers with orders")
+      .map(_.field1) // (orderKey, custKey, orderDate, shipPriority)
+      .withName("Project customer-order join product")
+
+      .join[(Long, Double), Long](_._1, lineItems, _._1)
+      .withName("Join CO with line items")
+      .map(coli => Query3Result(
+        orderKey = coli.field1._1,
+        revenue = coli.field1._2,
+        orderDate = coli.field0._3,
+        shipPriority = coli.field0._4
+      ))
+      .withName("Project CO-line-item join product")
+
+      .reduceByKey(
+        t => (t.orderKey, t.orderDate, t.shipPriority),
+        (t1, t2) => {
+          t1.revenue += t2.revenue;
+          t2
+        }
+      )
+      .withName("Aggregate revenue")
+      .collect()
+  }
+
+}
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/util/ExperimentDescriptor.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/util/ExperimentDescriptor.scala
new file mode 100644
index 0000000..32250af
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/util/ExperimentDescriptor.scala
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.util
+
+import org.apache.wayang.commons.util.profiledb.model.{Experiment, Subject}
+
+
+/**
+  * Implementing objects can be the [[Subject]] of a [[Experiment]].
+  */
+trait ExperimentDescriptor {
+
+  /**
+    * Retrieves the name of the experiment subject.
+    *
+    * @return the name
+    */
+  def name: String = this.getClass.getSimpleName.replace("$", "")
+
+  /**
+    * Retrieves the version of the experiment subject.
+    *
+    * @return the version
+    */
+  def version: String
+
+  /**
+    * The described [[Subject]].
+    */
+  def createSubject = new Subject(this.name, this.version)
+
+  /**
+    * Create a new [[Experiment]] for the described [[Subject]].
+    *
+    * @param id   the ID for the [[Experiment]]
+    * @param tags tags for the [[Experiment]]
+    * @return the [[Experiment]]
+    */
+  def createExperiment(id: String, tags: String*) = new Experiment(id, this.createSubject, tags: _*)
+
+}
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/util/Parameters.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/util/Parameters.scala
new file mode 100644
index 0000000..135c46a
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/util/Parameters.scala
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.util
+
+import org.apache.wayang.commons.util.profiledb.model.Experiment
+import org.apache.wayang.basic.WayangBasics
+import org.apache.wayang.core.optimizer.ProbabilisticDoubleInterval
+import org.apache.wayang.core.plugin.{DynamicPlugin, Plugin}
+//import org.apache.wayang.graphchi.GraphChi
+import org.apache.wayang.java.Java
+import org.apache.wayang.postgres.Postgres
+import org.apache.wayang.spark.Spark
+import org.apache.wayang.sqlite3.Sqlite3
+
+/**
+  * Utility to parse parameters of the apps.
+  */
+object Parameters {
+
+  private val yamlId = """yaml\((.*)\)""".r
+
+  val yamlPluginHel = "yaml(<YAML plugin URL>)"
+
+  private val intPattern = """[+-]?\d+""".r
+
+  private val longPattern = """[+-]?\d+L""".r
+
+  private val doublePattern = """[+-]?\d+\.\d*""".r
+
+  private val booleanPattern = """(?:true)|(?:false)""".r
+
+  private val probabilisticDoubleIntervalPattern = """(\d+)\.\.(\d+)(~\d+.\d+)?""".r
+
+  private val experiment =
+    """exp\(([^,;]+)(?:;tags=([^,;]+(?:,[^,;]+)*))?(?:;conf=([^,;:]+:[^,;:]+(?:,[^,;:]+:[^,;:]+)*))?\)""".r
+
+  val experimentHelp = "exp(<ID>[,tags=<tag>,...][,conf=<key>:<value>,...])"
+
+  /**
+    * Load a plugin.
+    *
+    * @param id name of the plugin
+    * @return the loaded [[Plugin]]
+    */
+  def loadPlugin(id: String): Plugin = id match {
+    case "basic-graph" => WayangBasics.graphPlugin
+    case "java" => Java.basicPlugin
+    case "java-graph" => Java.graphPlugin
+    case "java-conversions" => Java.channelConversionPlugin
+    case "spark" => Spark.basicPlugin
+    case "spark-graph" => Spark.graphPlugin
+    case "spark-conversions" => Spark.conversionPlugin
+//    case "graphchi" => GraphChi.plugin
+    case "postgres" => Postgres.plugin
+    case "postgres-conversions" => Postgres.conversionPlugin
+    case "sqlite3" => Sqlite3.plugin
+    case "sqlite3-conversions" => Sqlite3.conversionPlugin
+    case yamlId(url) => DynamicPlugin.loadYaml(url)
+    case other => throw new IllegalArgumentException(s"Could not load platform '$other'.")
+  }
+
+  /**
+    * Loads the specified [[Plugin]]s..
+    *
+    * @param platformIds a comma-separated list of platform IDs
+    * @return the loaded [[Plugin]]s
+    */
+  def loadPlugins(platformIds: String): Seq[Plugin] = loadPlugins(platformIds.split(","))
+
+  /**
+    * Loads the specified [[Plugin]]s.
+    *
+    * @param platformIds platform IDs
+    * @return the loaded [[Plugin]]s
+    */
+  def loadPlugins(platformIds: Seq[String]): Seq[Plugin] = platformIds.map(loadPlugin)
+
+  /**
+    * Create an [[Experiment]] for an experiment parameter and an [[ExperimentDescriptor]].
+    *
+    * @param experimentParameter  the parameter
+    * @param experimentDescriptor the [[ExperimentDescriptor]]
+    * @return the [[Experiment]]
+    */
+  def createExperiment(experimentParameter: String, experimentDescriptor: ExperimentDescriptor) =
+  experimentParameter match {
+    case experiment(id, tagList, confList) =>
+      val tags = tagList match {
+        case str: String => str.split(',').filterNot(_.isEmpty)
+        case _ => Array[String]()
+      }
+      val experiment = experimentDescriptor.createExperiment(id, tags: _*)
+      confList match {
+        case str: String => str.split(',').map(_.split(':')).foreach { pair =>
+          experiment.getSubject.addConfiguration(pair(0), parseAny(pair(1)))
+        }
+        case _ =>
+      }
+      experiment
+    case other => throw new IllegalArgumentException(s"Could parse experiment descriptor '$other'.")
+  }
+
+  /**
+    * Parses a given [[String]] into a specific basic type.
+    *
+    * @param str the [[String]]
+    * @return the parsed value
+    */
+  def parseAny(str: String): AnyRef = {
+    str match {
+      case "null" => null
+      case intPattern() => java.lang.Integer.valueOf(str)
+      case longPattern() => java.lang.Long.valueOf(str.take(str.length - 1))
+      case doublePattern() => java.lang.Double.valueOf(str)
+      case booleanPattern() => java.lang.Boolean.valueOf(str)
+      case probabilisticDoubleIntervalPattern(lower, upper, conf) =>
+        new ProbabilisticDoubleInterval(lower.toDouble, upper.toDouble, if (conf == null) 1d else conf.substring(1).toDouble)
+      case other: String => other
+    }
+  }
+
+}
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/util/ProfileDBHelper.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/util/ProfileDBHelper.scala
new file mode 100644
index 0000000..7c4b381
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/util/ProfileDBHelper.scala
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.util
+
+import org.apache.wayang.commons.util.profiledb.model.Experiment
+import org.apache.wayang.core.api.Configuration
+import org.apache.wayang.core.profiling.ProfileDBs
+
+import java.io.File
+
+/**
+  * Helper utility to employ with [[ProfileDB]].
+  */
+object ProfileDBHelper {
+
+  /**
+    * Stores the given [[Experiment]] if the [[Configuration]] defines a ProfileDB.
+    *
+    * @param experiment    the [[Experiment]]
+    * @param configuration [[Configuration]]
+    */
+  def store(experiment: Experiment, configuration: Configuration) = {
+    configuration.getStringProperty("wayang.apps.profiledb", null) match {
+      case path: String => {
+        println(s"Storing experiment '${experiment.getId}' to $path.")
+        //TODO: do the correction to store the element on the profile-db
+        //val profileDB = ProfileDBs.createProfileDB
+        //profileDB.append(new File(path), experiment)
+      }
+      case _ =>
+    }
+  }
+
+}
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/util/StdOut.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/util/StdOut.scala
new file mode 100644
index 0000000..4cc15a4
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/util/StdOut.scala
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.util
+
+import java.util.Objects
+
+/**
+  * Utility for printing to the stdout.
+  */
+object StdOut {
+
+  def printLimited[T](iterable: Iterable[T],
+                      limit: Int = 10,
+                      formatter: T => String = (t: T) => Objects.toString(t)): Unit = {
+    iterable.take(limit).map(formatter).foreach(println)
+    val numRemainders = iterable.size - limit
+    if (numRemainders > 0) {
+      println(f"...and $numRemainders%,d more.")
+    }
+  }
+
+}
diff --git a/wayang-benchmark/src/main/scala/org/apache/wayang/apps/wordcount/WordCountScala.scala b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/wordcount/WordCountScala.scala
new file mode 100644
index 0000000..722cf43
--- /dev/null
+++ b/wayang-benchmark/src/main/scala/org/apache/wayang/apps/wordcount/WordCountScala.scala
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.wordcount
+
+import org.apache.wayang.apps.util.{ExperimentDescriptor, Parameters, ProfileDBHelper}
+import org.apache.wayang.api._
+import org.apache.wayang.apps.util.ProfileDBHelper
+import org.apache.wayang.commons.util.profiledb.model.Experiment
+import org.apache.wayang.core.api.{Configuration, WayangContext}
+import org.apache.wayang.core.optimizer.ProbabilisticDoubleInterval
+import org.apache.wayang.core.plugin.Plugin
+import org.apache.wayang.core.util.fs.FileSystems
+
+/**
+  * This is app counts words in a file.
+  *
+  * @see [[Main]]
+  */
+class WordCountScala(plugin: Plugin*) {
+
+  /**
+    * Run the word count over a given file.
+    *
+    * @param inputUrl     URL to the file
+    * @param wordsPerLine optional estimate of how many words there are in each line
+    * @return the counted words
+    */
+  def apply(inputUrl: String,
+            wordsPerLine: ProbabilisticDoubleInterval = new ProbabilisticDoubleInterval(100, 10000, .8d))
+           (implicit configuration: Configuration, experiment: Experiment) = {
+    val wayangCtx = new WayangContext(configuration)
+    plugin.foreach(wayangCtx.register)
+    val planBuilder = new PlanBuilder(wayangCtx)
+
+    planBuilder
+      .withJobName(s"WordCount ($inputUrl)")
+      .withExperiment(experiment)
+      .withUdfJarsOf(this.getClass)
+      .readTextFile(inputUrl).withName("Load file")
+      .flatMap(_.split("\\W+"), selectivity = wordsPerLine).withName("Split words")
+      .filter(_.nonEmpty, selectivity = 0.99).withName("Filter empty words")
+      .map(word => (word.toLowerCase, 1)).withName("To lower case, add counter")
+      .reduceByKey(_._1, (c1, c2) => (c1._1, c1._2 + c2._2)).withName("Add counters")
+      .withCardinalityEstimator((in: Long) => math.round(in * 0.01))
+      .collect()
+  }
+
+}
+
+/**
+  * Companion object for [[WordCountScala]].
+  */
+object WordCountScala extends ExperimentDescriptor {
+
+  override def version = "0.1.0"
+
+  def main(args: Array[String]) {
+    // Parse args.
+    if (args.isEmpty) {
+      println(s"Usage: <main class> ${Parameters.experimentHelp} <plugin(,plugin)*> <input file> [<words per line a..b[~confidence]>]")
+      sys.exit(1)
+    }
+    implicit val configuration = new Configuration
+    implicit val experiment = Parameters.createExperiment(args(0), this)
+    val plugins = Parameters.loadPlugins(args(1))
+    experiment.getSubject.addConfiguration("plugins", args(1))
+    val inputFile = args(2)
+    experiment.getSubject.addConfiguration("input", inputFile)
+    val wordsPerLine = if (args.length >= 4) {
+      experiment.getSubject.addConfiguration("wordsPerLine", args(3))
+      Parameters.parseAny(args(3)).asInstanceOf[ProbabilisticDoubleInterval]
+    } else null
+
+    // Run wordCount.
+    val wordCount = new WordCountScala(plugins: _*)
+    val words =
+      (if (wordsPerLine != null) {
+        wordCount(inputFile, wordsPerLine)
+      } else {
+        wordCount(inputFile)
+      }).toSeq.sortBy(-_._2)
+
+    // Store experiment data.
+    val inputFileSize = FileSystems.getFileSize(inputFile)
+    if (inputFileSize.isPresent) experiment.getSubject.addConfiguration("inputSize", inputFileSize.getAsLong)
+    ProfileDBHelper.store(experiment, configuration)
+
+    // Print results.
+    println(s"Found ${words.size} words:")
+    words.take(10).foreach(wc => println(s"${wc._2}x ${wc._1}"))
+    if (words.size > 10) print(s"${words.size - 10} more...")
+  }
+
+}
diff --git a/wayang-benchmark/src/test/java/org/apache/wayang/apps/tpch/data/LineItemTupleTest.java b/wayang-benchmark/src/test/java/org/apache/wayang/apps/tpch/data/LineItemTupleTest.java
new file mode 100644
index 0000000..506a0cb
--- /dev/null
+++ b/wayang-benchmark/src/test/java/org/apache/wayang/apps/tpch/data/LineItemTupleTest.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.tpch.data;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.Calendar;
+import java.util.GregorianCalendar;
+
+/**
+ * Test suited for {@link LineItemTuple}.
+ */
+public class LineItemTupleTest {
+
+    @Test
+    public void testParser() {
+        LineItemTuple.Parser parser = new LineItemTuple.Parser();
+        final LineItemTuple tuple = parser.parse("\"3249925\";\"37271\";\"9775\";\"1\";\"9.00\";\"10874.43\";\"0.10\";" +
+                "\"0.04\";\"N\";\"O\";\"1998-04-19\";\"1998-06-17\";\"1998-04-21\";\"TAKE BACK RETURN         \";" +
+                "\"AIR       \";\"express instructions among the excuses nag\"");
+
+        Assert.assertEquals(3249925, tuple.L_ORDERKEY);
+        Assert.assertEquals(37271, tuple.L_PARTKEY);
+        Assert.assertEquals(9775, tuple.L_SUPPKEY);
+        Assert.assertEquals(1, tuple.L_LINENUMBER);
+        Assert.assertEquals(9.00, tuple.L_QUANTITY, 0);
+        Assert.assertEquals(10874.43, tuple.L_EXTENDEDPRICE, 0.001);
+        Assert.assertEquals(0.10, tuple.L_DISCOUNT, 0.001);
+        Assert.assertEquals(0.04, tuple.L_TAX, 0.001);
+        Assert.assertEquals('N', tuple.L_RETURNFLAG);
+        Assert.assertEquals('O', tuple.L_LINESTATUS);
+        Assert.assertEquals(this.toDateInteger(1998, 4, 19), tuple.L_SHIPDATE);
+        Assert.assertEquals(this.toDateInteger(1998, 6, 17), tuple.L_COMMITDATE);
+        Assert.assertEquals(this.toDateInteger(1998, 4, 21), tuple.L_RECEIPTDATE);
+        Assert.assertEquals("TAKE BACK RETURN         ", tuple.L_SHIPINSTRUCT);
+        Assert.assertEquals("AIR       ", tuple.L_SHIPMODE);
+        Assert.assertEquals("express instructions among the excuses nag", tuple.L_COMMENT);
+    }
+
+    private int toDateInteger(int year, int month, int date) {
+        final int[] months =new int[]{
+                Calendar.JANUARY, Calendar.FEBRUARY, Calendar.MARCH, Calendar.APRIL,
+                Calendar.MAY, Calendar.JUNE, Calendar.JULY, Calendar.AUGUST,
+                Calendar.SEPTEMBER, Calendar.OCTOBER, Calendar.NOVEMBER, Calendar.DECEMBER
+        };
+        Calendar calendar = GregorianCalendar.getInstance();
+        calendar.set(Calendar.YEAR, year);
+        calendar.set(Calendar.MONTH, months[month - 1]);
+        calendar.set(Calendar.DAY_OF_MONTH, date);
+        return (int) (calendar.getTimeInMillis() / (1000 * 60 * 60 * 24));
+    }
+
+}
diff --git a/wayang-benchmark/src/test/resources/kmeans-datagenerator.py b/wayang-benchmark/src/test/resources/kmeans-datagenerator.py
new file mode 100755
index 0000000..95ec76b
--- /dev/null
+++ b/wayang-benchmark/src/test/resources/kmeans-datagenerator.py
@@ -0,0 +1,56 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+#!/usr/bin/python
+
+import random
+
+class Distribution:
+    """Represents a normal distribution"""
+
+    def __init__(self, meanx, meany, stddev):
+        self.meanx = meanx
+        self.meany = meany
+        self.stddev = stddev
+
+    def sample(self):
+        x = random.gauss(self.meanx, self.stddev)
+        y = random.gauss(self.meany, self.stddev)
+        return x, y
+
+
+if __name__ == '__main__':
+    import sys
+    if len(sys.argv) == 1:
+        print "Usage: python datagen.py <#points> <avg x,avg y,standard dev>+"
+        sys.exit(1)
+
+    numpoints = int(sys.argv[1])
+
+    distributions = []
+    for arg in sys.argv[2:]:
+        parsedargs = map(float, arg.split(','))
+        distributions.append(Distribution(*parsedargs))
+
+    linecount = 0
+    while linecount < numpoints:
+        distribution = distributions[linecount % len(distributions)]
+        x, y = distribution.sample()
+        print '{:.5f},{:.5f}'.format(x, y)
+        linecount += 1
+
diff --git a/wayang-benchmark/src/test/resources/kmeans-k4-10000.input b/wayang-benchmark/src/test/resources/kmeans-k4-10000.input
new file mode 100644
index 0000000..c952430
--- /dev/null
+++ b/wayang-benchmark/src/test/resources/kmeans-k4-10000.input
@@ -0,0 +1,10000 @@
+-9.79368,-10.33969
+-9.01481,10.67542
+10.97226,10.40678
+11.26019,-10.84876
+-8.14145,-10.41136
+-10.34589,11.03210
+9.96483,12.42198
+8.93666,-9.17467
+-9.38376,-9.39328
+-8.93631,7.45452
+9.07177,9.73944
+9.92201,-9.66590
+-8.24793,-10.22278
+-10.56606,10.24368
+10.15560,10.75286
+10.59716,-10.83735
+-7.84170,-9.36725
+-9.87402,11.89988
+11.13649,9.85499
+10.16892,-8.13950
+-12.47831,-9.74064
+-11.02132,11.23068
+8.55119,8.69454
+11.14613,-10.59749
+-9.69827,-10.27026
+-11.20531,9.88907
+8.60972,9.36129
+9.43058,-10.77007
+-9.71563,-9.97020
+-10.51267,11.68758
+9.31615,10.95763
+9.56172,-11.37319
+-10.71710,-10.34925
+-9.21029,10.40779
+9.19540,8.44997
+8.30283,-8.07073
+-11.02906,-8.52147
+-11.43017,8.81139
+9.37590,10.36659
+10.48523,-10.67178
+-8.65379,-10.88288
+-9.40159,8.69212
+10.50783,10.72069
+10.58515,-11.66304
+-10.57392,-10.32520
+-9.94178,8.29523
+7.99990,11.45683
+10.64674,-10.33134
+-9.78962,-9.25584
+-10.10794,10.47778
+10.84404,9.13095
+9.26397,-11.88449
+-10.47652,-10.99625
+-7.92912,10.01446
+10.73704,9.04041
+9.14711,-10.00911
+-9.80601,-10.00780
+-10.19383,9.98950
+7.85893,9.05468
+9.70806,-11.56170
+-10.19743,-9.13079
+-7.89262,10.71490
+9.99844,10.21605
+11.07657,-9.98510
+-10.85458,-10.03756
+-8.96310,10.59544
+11.53066,11.75509
+10.83401,-10.10261
+-10.20899,-10.68434
+-9.25919,10.42435
+10.14935,8.71346
+9.68426,-8.98940
+-9.65356,-9.65171
+-11.62289,9.17878
+9.52741,9.90849
+10.22009,-11.40003
+-11.74540,-9.38791
+-11.25725,8.28476
+10.13707,11.45092
+10.43851,-9.78186
+-9.56708,-9.03213
+-11.43200,7.85401
+10.06104,8.39687
+9.67874,-9.65995
+-11.19041,-10.26291
+-9.58212,10.55541
+10.74544,10.45275
+10.11517,-9.76198
+-10.34722,-9.94784
+-8.53862,9.47782
+11.32659,9.26848
+9.35859,-10.57340
+-9.44107,-9.42337
+-11.17584,9.84249
+9.18927,9.34522
+10.59352,-11.07020
+-12.31806,-12.54431
+-8.87590,9.59110
+9.01668,9.50275
+9.81444,-8.13291
+-11.04900,-8.72454
+-9.37162,8.80156
+12.04248,9.98562
+10.02422,-9.86532
+-10.35167,-10.60433
+-10.44869,11.43521
+10.45359,9.88557
+10.62472,-9.47767
+-8.91936,-10.36097
+-10.95076,9.44153
+8.94166,10.51363
+10.73483,-10.14444
+-11.72768,-10.86415
+-11.59476,9.75459
+8.95532,11.38422
+10.96531,-8.70426
+-8.56877,-11.65072
+-11.03596,9.30226
+12.26249,10.53257
+8.17253,-8.70186
+-9.72402,-10.08005
+-9.31164,9.56294
+9.67040,10.77632
+8.85848,-10.99947
+-9.32906,-9.29728
+-10.78485,9.94476
+9.14522,9.61032
+9.50409,-10.68458
+-10.76379,-9.93284
+-10.77074,9.81951
+9.99543,11.90421
+10.53560,-9.99641
+-8.38603,-11.14237
+-9.97025,8.96908
+9.77237,11.48914
+9.53607,-8.90007
+-10.97721,-12.11405
+-10.26382,9.49790
+10.28203,10.60287
+10.87772,-10.71397
+-8.63492,-8.71783
+-9.43917,9.22359
+11.53473,9.23903
+9.37175,-11.90374
+-10.37353,-8.70421
+-8.83894,9.23652
+8.29348,9.61528
+11.51892,-10.23968
+-10.37646,-10.67046
+-10.56318,9.54243
+9.45123,10.45351
+10.03363,-10.31341
+-11.56748,-10.33620
+-7.94062,8.78666
+9.65578,10.93507
+10.93605,-10.30754
+-10.34710,-9.79927
+-10.90946,8.89621
+10.20292,9.43235
+10.43728,-8.63930
+-10.60280,-10.80840
+-11.00684,10.53015
+9.09425,9.42394
+10.58519,-10.91890
+-9.14363,-9.09864
+-10.64337,9.25751
+9.98680,10.72968
+12.26665,-7.65202
+-10.13647,-9.67893
+-9.28953,11.70167
+11.47781,9.70321
+10.23702,-10.82650
+-9.70676,-8.17677
+-9.47326,10.88824
+9.56785,10.99870
+10.90161,-10.58770
+-10.21035,-9.67172
+-11.27813,10.30866
+11.56476,9.46330
+10.02349,-10.15066
+-10.13818,-8.57611
+-8.43364,10.71884
+9.38803,9.62395
+8.93285,-9.15760
+-11.50689,-10.51620
+-10.08368,9.11029
+10.73742,9.97040
+10.60772,-9.36162
+-9.73345,-8.38344
+-11.64434,9.07765
+8.73066,11.72031
+9.51705,-10.98391
+-10.91285,-10.41028
+-9.46485,9.36468
+12.08836,9.89051
+9.48431,-10.94799
+-10.49742,-8.21095
+-12.62295,10.69693
+8.50012,11.48113
+10.33159,-11.82440
+-10.11554,-8.88958
+-9.97226,8.16094
+10.55584,9.83888
+10.78962,-8.82051
+-9.94726,-10.21734
+-8.11348,10.16695
+8.33710,8.92579
+9.40574,-9.43021
+-9.29692,-8.81960
+-12.55167,9.74967
+9.16718,9.90984
+10.56944,-10.63228
+-10.31270,-9.77850
+-10.05856,9.85324
+8.87166,9.77107
+11.45965,-9.01790
+-10.93046,-11.27764
+-10.57920,11.44904
+8.74315,9.86494
+10.53246,-12.46997
+-9.72752,-11.25554
+-9.45733,9.38062
+9.18327,10.97880
+8.84477,-10.83433
+-10.70236,-9.94738
+-10.18656,9.39179
+9.61978,10.08120
+8.91064,-9.99071
+-9.19537,-11.58961
+-8.23987,9.04220
+10.13216,10.65558
+11.84255,-9.51851
+-12.02947,-11.20503
+-9.01738,9.78232
+8.52412,9.22760
+9.23206,-10.01934
+-10.08313,-10.47049
+-10.42959,9.92465
+10.53694,9.42547
+9.92011,-9.28312
+-11.22084,-12.10561
+-10.06458,10.27362
+8.74699,9.54402
+9.54106,-10.74073
+-9.73654,-11.71973
+-10.82669,10.46839
+9.33761,8.98775
+11.38749,-11.18586
+-11.69845,-9.48238
+-8.50728,10.76630
+9.02174,9.50043
+8.80498,-9.37208
+-9.64009,-9.04399
+-10.15852,10.43781
+11.36748,11.17149
+9.43230,-10.72387
+-10.52921,-9.40145
+-10.84655,10.05523
+8.57054,10.49726
+10.56092,-11.27388
+-11.66179,-8.29416
+-8.45190,10.33657
+9.89799,9.49720
+10.74396,-10.62908
+-9.99788,-10.15437
+-9.12874,10.17153
+8.94960,9.83331
+9.98423,-9.53272
+-7.73535,-10.60975
+-11.22678,11.47562
+9.89659,9.85908
+9.63412,-10.77088
+-12.09358,-10.72955
+-9.20801,10.85838
+9.65129,9.86136
+8.91581,-8.86863
+-9.15092,-9.98016
+-8.54279,13.05384
+10.33995,11.08811
+8.79732,-10.34331
+-10.32836,-8.93466
+-10.66930,11.66697
+12.17489,9.32798
+9.73557,-10.05367
+-10.99285,-11.28045
+-10.58727,9.43534
+6.83562,12.37422
+9.27901,-7.96993
+-8.15666,-9.00697
+-11.53272,10.17777
+9.19620,10.08400
+8.51515,-9.64129
+-8.76971,-9.70588
+-8.73129,9.42067
+11.46698,10.54839
+10.97356,-11.11716
+-10.18037,-9.05750
+-9.97947,9.60300
+9.84139,10.96636
+10.91673,-9.69621
+-9.33751,-9.65364
+-7.74271,10.58380
+11.68461,9.17622
+10.34523,-7.54950
+-12.44410,-10.61954
+-9.23390,7.78509
+9.57294,10.00417
+9.37284,-8.73329
+-8.87914,-9.06952
+-9.01943,10.12694
+9.54307,10.05962
+8.82202,-8.95999
+-10.19576,-8.74917
+-9.86135,8.83903
+10.30872,9.95214
+10.20653,-8.08634
+-10.10938,-9.85353
+-8.78177,11.02709
+8.22217,9.49148
+10.39017,-10.99444
+-9.28398,-10.66285
+-8.64915,10.95893
+9.97280,10.90902
+10.35458,-10.29052
+-9.47240,-11.41833
+-10.65803,9.69750
+9.78191,8.81993
+10.62777,-8.80649
+-9.91081,-10.75363
+-9.19023,9.72844
+10.33562,9.62697
+9.92252,-11.82226
+-11.30095,-10.46239
+-8.75405,8.87864
+9.21609,9.05434
+9.25850,-10.57769
+-10.19779,-8.89684
+-12.10796,11.49167
+8.93151,10.67034
+9.39658,-9.40967
+-10.14566,-9.37502
+-11.45299,10.30867
+9.57986,11.27340
+9.71155,-10.39085
+-11.07733,-10.49844
+-10.32675,10.21853
+9.79237,11.27381
+9.72685,-8.43924
+-10.09357,-8.86067
+-10.22098,12.21750
+10.55711,8.92324
+8.56406,-8.59608
+-9.71234,-10.41451
+-10.57421,10.07813
+9.80725,12.72269
+10.02260,-10.21056
+-8.02256,-10.30514
+-10.06765,7.95404
+9.94959,9.60977
+9.87498,-10.55408
+-9.76269,-11.03776
+-9.11297,10.90960
+10.02814,9.67430
+9.67515,-10.11896
+-7.82527,-11.63241
+-11.14461,8.61688
+11.34390,11.10661
+9.32201,-10.01200
+-10.53094,-11.08854
+-11.33301,8.77759
+9.51018,10.94821
+10.33486,-10.50028
+-10.27515,-10.93643
+-10.49215,9.34186
+10.40525,10.29121
+10.74338,-9.19679
+-9.32220,-10.48210
+-9.89847,9.48042
+9.42634,10.34067
+11.46765,-9.31513
+-9.68237,-10.27066
+-9.72750,9.53789
+11.55013,8.88352
+8.29257,-8.99292
+-11.19131,-8.83869
+-9.73316,10.99004
+9.71769,9.33291
+11.72677,-11.74381
+-9.33609,-9.96329
+-9.46912,10.01758
+10.31076,9.55801
+11.92718,-8.84747
+-9.19758,-8.99248
+-9.80238,8.88843
+8.77952,11.15153
+9.09775,-10.64970
+-10.32376,-9.95137
+-10.44653,9.03468
+8.88586,9.86818
+10.83230,-8.92979
+-10.95206,-8.44987
+-10.49795,8.86211
+10.76539,9.75626
+10.05187,-9.95252
+-10.59451,-10.36601
+-9.50168,8.54477
+9.02324,8.73459
+10.86232,-10.86776
+-10.46570,-9.83258
+-10.36175,9.78458
+8.86570,10.46702
+9.67371,-9.78549
+-7.97268,-11.18148
+-10.15204,10.72920
+10.19498,10.91335
+11.56229,-9.57819
+-8.33450,-9.35600
+-10.17935,9.03511
+10.72525,10.68764
+9.68665,-8.93105
+-9.86881,-11.17745
+-10.01785,9.85199
+8.34920,9.21537
+10.48049,-9.00267
+-10.74118,-10.28372
+-10.97112,10.03576
+10.34018,10.30267
+9.54398,-7.76596
+-10.19873,-9.68794
+-9.56187,8.67420
+9.21173,11.12718
+9.19640,-11.39304
+-9.99593,-9.02433
+-10.29247,10.81715
+10.22882,10.90142
+10.41075,-10.17996
+-9.89742,-9.96130
+-9.03147,9.37920
+8.76353,11.02321
+9.66909,-10.13127
+-9.29726,-9.05707
+-9.86061,11.84215
+9.31960,9.88601
+10.10185,-9.61556
+-9.73061,-9.07528
+-8.47390,9.86567
+11.19004,11.01971
+10.66315,-8.04561
+-10.68906,-11.48715
+-10.95280,8.15656
+9.29656,9.60491
+10.67042,-8.64692
+-9.93203,-9.88287
+-10.20535,10.58112
+9.38143,8.64947
+9.35566,-9.67625
+-9.12006,-10.16932
+-10.78444,11.03323
+10.11822,10.23338
+9.17640,-9.86295
+-9.77921,-10.04492
+-10.74912,9.94050
+11.43458,11.09887
+10.65585,-10.47783
+-10.69287,-9.65328
+-10.72368,10.10807
+10.73174,10.30144
+11.78691,-9.40420
+-10.15969,-10.04571
+-9.10695,11.18534
+11.01437,10.27331
+11.28239,-9.86605
+-9.89278,-10.47345
+-10.08280,10.55130
+9.51257,12.66379
+7.93429,-9.49205
+-10.55586,-11.25028
+-10.28550,11.21294
+12.18008,9.30427
+10.35591,-10.00560
+-9.62444,-9.16509
+-10.27604,11.00706
+9.61073,10.36701
+9.90518,-9.48443
+-11.32875,-11.09972
+-10.96878,9.63767
+12.73593,9.90963
+10.57013,-8.57780
+-9.74991,-9.70230
+-8.56759,9.17781
+8.78794,9.83569
+8.77806,-9.32459
+-9.62807,-10.38914
+-9.84343,9.44392
+10.70915,10.21462
+8.58604,-9.90271
+-10.03390,-8.27417
+-8.18247,8.88746
+11.04306,9.38408
+9.89084,-10.54792
+-10.92345,-9.84399
+-9.39771,7.96586
+9.90346,11.59189
+8.57412,-10.62984
+-9.36897,-10.50535
+-10.00145,9.20474
+9.75208,9.04522
+9.16725,-8.91886
+-9.58552,-10.16052
+-8.08489,8.75810
+9.61103,10.46085
+8.04863,-10.31457
+-9.38382,-9.16755
+-10.91852,10.93458
+12.21548,9.54477
+12.14916,-10.10820
+-10.20002,-9.39063
+-10.96230,10.34070
+9.40486,11.11467
+11.64385,-10.37174
+-10.94801,-11.04492
+-10.05886,9.20833
+8.84682,9.50986
+9.99775,-10.89952
+-11.62132,-8.67580
+-10.36358,10.99897
+10.70864,10.02657
+9.92213,-10.70652
+-11.10392,-10.16272
+-8.56298,9.99984
+8.11087,10.44338
+9.22566,-9.05411
+-9.92700,-9.53057
+-10.45012,10.46915
+10.52545,12.06182
+9.43556,-9.13965
+-10.08529,-8.52611
+-9.65546,8.23038
+11.44729,11.63721
+10.82532,-9.37709
+-11.16912,-9.98406
+-11.10756,10.87850
+9.79423,11.31156
+10.55254,-8.10163
+-10.39644,-10.89666
+-11.49041,10.28740
+10.34589,9.44536
+10.91092,-10.06585
+-12.29877,-11.08361
+-9.26609,10.97262
+10.57528,9.15431
+11.14703,-10.46930
+-9.95581,-9.05772
+-10.66585,9.68681
+8.93464,9.05756
+8.94152,-8.85031
+-9.65915,-9.37587
+-9.72908,11.20511
+8.45814,9.71875
+8.07422,-11.10305
+-10.04896,-10.65676
+-9.37836,11.59122
+8.44853,8.61357
+9.69478,-8.74517
+-9.27906,-9.12467
+-11.18986,10.34695
+10.69575,12.66607
+12.11282,-11.31843
+-10.85502,-10.69573
+-10.40430,9.32698
+8.27763,10.72346
+9.94457,-8.22343
+-11.35086,-8.80354
+-10.86976,9.13435
+9.46539,9.92171
+9.33387,-12.03179
+-11.09834,-7.71893
+-9.75933,8.65769
+9.11884,9.67410
+11.52145,-10.69716
+-8.02340,-10.28111
+-10.39149,8.47735
+9.46138,10.04147
+8.45221,-10.62723
+-11.05689,-7.99241
+-9.27582,10.35591
+11.30457,10.41720
+9.81467,-8.54029
+-10.72627,-9.94148
+-11.11578,9.14631
+8.55019,9.67726
+8.51455,-10.07964
+-10.06574,-9.71330
+-9.92299,10.36476
+11.06057,10.14668
+9.98783,-10.32396
+-9.83704,-9.07064
+-9.49763,9.30969
+9.71681,9.10934
+9.18424,-10.56584
+-10.86868,-11.77171
+-10.58772,10.12393
+9.76188,11.36424
+8.68052,-10.73861
+-10.19916,-9.87406
+-9.75501,10.65906
+11.16836,10.20552
+10.02929,-11.83835
+-9.00604,-10.63147
+-9.38372,9.40904
+8.92810,9.82189
+10.86453,-11.43496
+-9.03552,-11.16091
+-10.39862,9.99462
+11.49918,9.28239
+9.13817,-10.43962
+-9.79798,-11.46637
+-10.07962,10.18008
+9.80135,10.05981
+10.25233,-8.82844
+-10.94970,-8.65639
+-9.76842,8.32931
+9.42845,10.25179
+9.51918,-9.66519
+-9.90615,-10.38190
+-10.10920,8.43187
+9.21761,9.00380
+10.80829,-9.85384
+-8.49478,-9.93104
+-11.48293,9.98014
+8.90314,9.30530
+10.26949,-10.66896
+-10.57072,-9.60951
+-11.40263,10.94646
+10.46624,10.96272
+10.30560,-11.77037
+-9.45761,-10.54830
+-11.75708,8.15486
+10.40330,9.74741
+9.98124,-10.91447
+-10.17522,-10.05612
+-9.49177,11.44310
+10.56986,8.99018
+9.86859,-10.03142
+-10.92112,-8.81588
+-8.99562,9.85834
+10.70318,10.77252
+8.92720,-8.41365
+-9.64756,-9.32352
+-10.54155,10.32172
+10.98415,11.03887
+9.35610,-9.51737
+-10.20970,-8.23472
+-10.53000,9.47445
+8.94877,8.46070
+10.85133,-9.38922
+-11.68983,-10.75368
+-9.09087,11.13640
+11.46312,10.32851
+10.39715,-9.08676
+-7.48765,-10.78432
+-8.25649,9.63380
+11.29127,10.31621
+9.32324,-9.64540
+-9.59100,-10.19899
+-9.31506,11.23049
+10.79775,10.98465
+9.74598,-9.39447
+-9.99568,-11.03432
+-9.61136,11.35167
+8.59515,10.15253
+10.81602,-9.38945
+-8.60093,-10.43504
+-13.10893,12.27682
+10.74545,10.20323
+10.58382,-10.90834
+-9.33039,-11.48314
+-10.08813,10.60387
+9.52143,9.61156
+9.74824,-10.17096
+-9.62152,-10.20172
+-10.31581,9.98286
+10.63100,9.36161
+9.87603,-9.54724
+-8.58422,-9.84887
+-10.83405,10.80902
+10.72438,9.52603
+11.29309,-9.64493
+-12.30092,-9.72101
+-10.11353,10.16459
+10.72285,8.83688
+11.26838,-8.86400
+-10.70658,-8.68274
+-10.28189,8.48556
+8.73866,9.70065
+10.70193,-9.88340
+-9.80924,-7.61104
+-10.73744,9.63204
+10.07034,10.57543
+7.89050,-10.57092
+-12.14997,-9.31281
+-9.94708,10.88726
+10.20385,10.13759
+9.31741,-9.33821
+-11.62894,-10.30213
+-10.26419,10.06877
+10.41083,9.78978
+9.28793,-11.17345
+-11.04307,-9.27921
+-9.18702,10.88002
+10.50606,8.54755
+9.65278,-10.82138
+-9.79236,-9.54056
+-10.84868,10.83925
+11.01584,9.45270
+10.37738,-9.54074
+-9.62451,-11.95868
+-10.49274,11.19160
+9.60452,9.92080
+8.16588,-10.54932
+-8.38224,-11.34540
+-8.51619,11.00311
+9.34138,7.21700
+7.54427,-11.18990
+-9.83042,-9.97601
+-11.22177,7.27315
+10.84508,10.41795
+9.09130,-11.21620
+-9.12325,-8.09637
+-8.51778,10.20332
+10.63886,11.33732
+11.16018,-9.21811
+-9.00860,-10.68216
+-9.25522,11.43396
+10.60485,12.13877
+9.40756,-9.93142
+-9.04219,-10.73700
+-10.52487,11.92262
+10.37275,10.33754
+11.17629,-8.10993
+-8.66148,-11.00488
+-9.91767,7.91318
+11.47015,8.19186
+9.96854,-10.32496
+-9.59635,-9.81856
+-11.26042,10.52113
+9.05923,10.09819
+10.14873,-9.45789
+-9.85936,-9.88665
+-8.05473,9.18591
+9.70685,8.11573
+10.04038,-10.56604
+-10.74232,-11.18622
+-11.68410,8.56021
+8.47158,10.45066
+10.47433,-11.18132
+-8.22640,-9.05240
+-10.26154,9.91835
+8.11316,10.58485
+10.85743,-8.96598
+-8.22680,-11.44395
+-10.63857,10.42439
+9.89690,12.01774
+10.22298,-8.98401
+-10.33931,-10.05790
+-9.06899,8.89475
+11.18281,11.28170
+10.77000,-8.77456
+-9.79401,-10.13130
+-11.17628,9.70145
+9.87145,9.70377
+9.69762,-10.84473
+-8.82593,-11.82533
+-10.24083,10.29163
+10.77496,11.12652
+9.96398,-10.58660
+-11.22447,-10.62531
+-10.00902,9.65674
+11.62963,10.28127
+8.93218,-9.37134
+-12.03085,-10.65435
+-8.00727,9.50414
+8.36920,7.12462
+9.10135,-10.32746
+-9.06153,-9.85937
+-9.70887,8.75704
+11.38188,8.93497
+10.59588,-11.25423
+-9.34265,-9.72583
+-8.99802,8.96843
+9.87959,9.34446
+9.03480,-10.80165
+-10.77452,-10.88067
+-8.82629,8.95488
+8.82793,10.51206
+11.54589,-10.17588
+-8.70462,-11.04081
+-10.60388,10.40642
+10.22321,10.91759
+8.37684,-10.23113
+-11.84697,-9.84955
+-9.83026,10.06308
+11.30310,9.12814
+9.07262,-10.81081
+-9.38117,-10.24443
+-9.79438,9.61188
+9.90684,11.05221
+11.57658,-9.96895
+-10.43001,-11.60107
+-10.44188,9.61288
+10.45595,10.20061
+12.58704,-10.35770
+-9.45554,-10.29455
+-11.55499,10.14598
+9.47437,10.72074
+11.22283,-10.78930
+-9.64017,-9.83642
+-11.87168,9.03387
+8.70408,9.39427
+9.49389,-7.23825
+-10.32561,-9.60070
+-11.21001,11.57710
+10.31587,9.30129
+10.49158,-10.79861
+-10.88357,-10.05640
+-10.69245,8.99771
+10.75833,10.13672
+10.04144,-8.75615
+-10.41899,-10.10826
+-8.52337,10.87852
+10.55645,9.11607
+12.17868,-9.87773
+-9.09834,-9.80341
+-9.90715,9.40425
+9.89200,9.19273
+11.37935,-9.30256
+-8.70903,-11.67022
+-11.02514,10.49985
+10.50413,10.78348
+8.57929,-8.30915
+-10.43893,-10.33262
+-9.06159,9.08668
+10.30111,10.87867
+9.98154,-9.38019
+-9.76794,-10.00974
+-9.02821,9.37006
+8.74814,8.86197
+12.02133,-10.63620
+-9.91797,-9.13008
+-9.97484,10.00541
+9.33758,10.99169
+9.05678,-11.41252
+-9.57823,-9.97783
+-10.13036,8.59673
+8.70496,9.49154
+10.73862,-9.32952
+-10.83531,-10.28959
+-11.70593,9.46696
+9.93741,11.61813
+6.84601,-8.49788
+-10.33630,-9.67379
+-9.72542,9.67934
+11.39247,10.87464
+10.12826,-11.08359
+-9.28641,-8.94633
+-10.15052,10.94853
+10.08792,9.70716
+9.90793,-10.26288
+-10.36005,-9.61522
+-9.59180,12.55985
+9.67820,9.25429
+10.86429,-9.37558
+-11.75790,-10.21468
+-7.52408,8.31942
+9.36296,9.23815
+11.54760,-10.72985
+-9.55261,-10.36757
+-9.51075,10.36988
+10.45029,10.98436
+10.04390,-9.48174
+-10.68476,-10.82453
+-10.65740,8.64566
+9.16580,10.05692
+11.55856,-8.13253
+-8.74079,-9.71977
+-10.57726,9.22751
+11.64581,9.33151
+10.77306,-8.67118
+-10.88872,-10.50682
+-9.30666,11.41795
+10.79592,8.92337
+12.03515,-9.92105
+-11.11444,-9.81871
+-9.38316,9.46387
+9.60294,8.55944
+10.65103,-11.16370
+-7.90700,-10.35961
+-10.28617,9.68380
+8.18584,9.78759
+10.24893,-9.47368
+-9.00314,-11.27492
+-9.38268,8.63299
+11.61394,10.48509
+10.84424,-11.61034
+-11.64418,-10.28916
+-9.44718,10.76521
+9.62598,11.44401
+12.14217,-8.90493
+-9.42861,-8.57716
+-8.16756,9.02347
+9.29111,9.89321
+9.42551,-8.69101
+-10.52498,-9.41554
+-9.53451,10.71589
+11.04231,9.40389
+11.49068,-10.81254
+-8.54443,-9.22418
+-11.97188,9.41307
+9.61121,11.33713
+11.37512,-10.60136
+-9.16545,-10.43451
+-10.56629,9.40350
+9.81261,8.98762
+9.58756,-9.74324
+-9.51952,-8.59369
+-9.99422,11.26312
+10.15055,10.22420
+9.88916,-9.32447
+-10.80135,-10.80949
+-9.83329,11.38808
+7.00574,9.52934
+9.93652,-8.86457
+-9.89047,-9.96752
+-8.99647,9.81075
+9.93349,9.04509
+11.19247,-10.13389
+-10.65578,-10.00760
+-10.55640,10.80903
+9.07993,10.45414
+9.21815,-9.68653
+-11.23986,-9.56842
+-9.97089,9.00608
+10.62119,9.41245
+9.07663,-10.33238
+-10.51422,-10.37324
+-9.15537,8.62085
+10.24429,8.64940
+10.02696,-9.86443
+-9.00875,-9.91910
+-11.47409,10.29052
+8.48461,10.58233
+9.43861,-10.66236
+-9.50365,-9.80464
+-8.23168,9.26772
+8.76942,10.29316
+10.81347,-11.62716
+-8.96172,-9.92683
+-10.38282,10.38008
+8.52722,10.07692
+9.48252,-9.85589
+-10.64286,-9.49615
+-9.80082,10.72467
+9.75479,9.00436
+10.76051,-11.12796
+-8.49339,-8.27471
+-9.53979,8.93691
+10.11639,9.72378
+9.23321,-8.36265
+-9.75841,-10.96584
+-10.55026,10.49143
+8.83101,8.83853
+9.46422,-7.74007
+-8.62762,-10.29011
+-9.56560,9.66920
+10.67223,9.06767
+9.96580,-8.98481
+-7.99086,-10.32011
+-11.10038,9.56025
+10.82277,11.76546
+8.98882,-10.71730
+-8.07074,-10.23730
+-10.33488,11.65217
+8.86675,9.99378
+10.34460,-8.31878
+-9.70714,-9.34846
+-9.48924,9.78467
+10.70953,11.03321
+9.11485,-8.93791
+-8.34464,-10.83223
+-9.54659,10.52113
+8.44380,9.41465
+11.34642,-9.85174
+-9.46350,-10.26398
+-9.63235,10.21173
+9.57043,9.90245
+9.82349,-9.69968
+-9.09697,-12.18039
+-9.58079,10.60897
+9.75815,9.62448
+9.00139,-8.74343
+-8.44118,-9.67146
+-10.41742,10.36962
+9.70019,10.58746
+11.10681,-9.32755
+-8.61362,-10.17514
+-11.60160,10.92496
+8.35795,9.98121
+9.82336,-9.76604
+-9.62955,-11.54375
+-10.34248,10.94135
+10.33609,10.50790
+10.72128,-10.02837
+-9.01084,-8.94938
+-8.46249,10.48666
+10.46472,10.51589
+9.27275,-11.35306
+-10.63874,-9.97620
+-10.59494,11.94846
+8.13587,8.46642
+10.72134,-10.16717
+-10.08625,-11.02652
+-11.74678,9.51489
+9.10484,10.41464
+7.84714,-10.73398
+-10.66789,-8.24442
+-11.72196,10.20691
+10.80826,8.31064
+8.64067,-9.46564
+-10.17341,-10.95776
+-10.21158,9.10591
+11.08066,8.72123
+11.27906,-10.37018
+-11.14202,-9.10923
+-9.10950,10.03021
+10.51865,10.71446
+11.06706,-11.30716
+-8.37354,-9.91922
+-10.64358,10.98503
+8.72748,7.84549
+10.40725,-10.20486
+-8.10880,-10.29868
+-10.00705,8.37629
+8.92166,9.91644
+11.15175,-10.04169
+-10.71470,-9.96807
+-9.55171,12.13015
+10.32438,10.72803
+9.19947,-10.99510
+-10.96082,-9.64098
+-11.74303,9.35461
+9.27871,11.02560
+10.86773,-9.69810
+-9.82380,-9.90927
+-10.88722,10.01872
+8.28087,10.40642
+9.60430,-8.67110
+-10.22487,-12.23975
+-9.95967,10.45764
+10.46440,9.81909
+11.25868,-9.58204
+-8.84307,-9.27104
+-11.77432,10.33360
+9.31424,8.89760
+8.32509,-9.07151
+-11.77383,-11.02100
+-10.62874,8.76016
+9.25417,10.91065
+9.16453,-10.83076
+-11.07701,-9.37987
+-11.68571,11.32036
+9.26779,8.79553
+10.48816,-9.85092
+-11.00500,-10.44500
+-10.77939,11.23796
+10.76973,8.88317
+10.12887,-11.93170
+-8.87182,-11.81942
+-10.65860,9.39269
+11.42868,9.71786
+11.84576,-11.21901
+-10.53119,-10.14811
+-10.22991,10.41637
+9.90129,9.88248
+8.79100,-9.62579
+-10.22694,-9.75386
+-9.43495,10.04200
+10.48670,10.62147
+9.97463,-8.43108
+-10.79143,-10.65008
+-10.25945,9.02519
+11.11637,10.08720
+8.45265,-8.81695
+-9.74117,-11.29211
+-10.71580,11.26205
+11.96391,11.51059
+9.27831,-9.50923
+-8.30859,-12.31908
+-8.94568,10.80171
+10.22776,8.75060
+10.14367,-11.97924
+-10.28835,-10.42641
+-10.44348,11.12586
+10.30136,10.25497
+8.43828,-10.08917
+-8.50822,-11.19757
+-9.70550,10.12482
+10.03495,11.18769
+10.68954,-8.88723
+-10.56412,-11.13264
+-10.47492,10.99877
+9.13959,9.17511
+9.19461,-10.49931
+-10.94526,-10.51600
+-9.67143,9.27999
+10.05558,10.10765
+10.61413,-9.07423
+-10.25876,-10.27316
+-9.30369,11.47735
+9.59134,7.79537
+9.99754,-10.36831
+-8.59446,-11.47810
+-9.75429,10.63832
+8.70661,11.07982
+10.97103,-9.67406
+-8.60218,-9.22132
+-9.19330,9.96846
+10.88017,8.63509
+10.40521,-9.03197
+-10.53707,-9.57427
+-10.38467,10.49326
+9.38730,10.10441
+11.72095,-9.28287
+-10.77916,-9.80895
+-9.33767,9.89647
+8.66138,6.97491
+10.52452,-12.20705
+-11.43817,-9.38780
+-10.00824,10.52373
+9.18641,11.81558
+9.27953,-10.28049
+-7.81403,-11.11194
+-7.85385,10.19309
+11.88546,10.44669
+8.60937,-9.14285
+-8.38734,-10.09819
+-9.57950,7.38057
+8.40502,10.48746
+10.83582,-8.50395
+-10.00391,-10.02146
+-10.64597,10.61367
+9.29179,10.83099
+10.16195,-7.50221
+-10.65286,-10.69036
+-9.05803,8.70708
+9.30395,10.83265
+8.81320,-10.26722
+-10.45016,-9.66306
+-9.53091,9.64057
+11.73475,9.20666
+7.96738,-9.93757
+-7.80677,-10.44873
+-8.73205,8.92118
+9.43315,10.38067
+9.72670,-9.92341
+-10.01159,-9.78196
+-10.65079,9.43530
+8.75790,10.14818
+9.26784,-10.81852
+-9.74065,-10.18449
+-9.50802,8.66310
+11.71213,10.76173
+10.64146,-10.09337
+-10.66130,-10.06299
+-9.10446,9.46300
+9.97569,9.71264
+8.94097,-10.50381
+-11.03986,-10.61647
+-8.34478,9.41709
+9.29752,10.47366
+10.69087,-9.33059
+-9.87099,-6.13960
+-12.22594,9.71129
+9.74995,11.20134
+10.10613,-12.06895
+-8.95429,-9.84839
+-9.37360,10.25669
+10.19071,10.20419
+9.91934,-8.14780
+-10.87587,-8.94725
+-9.44046,11.02737
+11.11932,9.48287
+8.95535,-9.93754
+-11.55678,-9.90794
+-9.17386,9.56168
+9.02399,9.31670
+10.02604,-9.82100
+-10.24357,-10.07398
+-10.24324,10.73988
+9.64649,8.81869
+10.34763,-9.97297
+-9.62016,-9.61361
+-10.09509,10.11561
+8.36587,9.90437
+8.55620,-9.75440
+-10.23334,-10.15295
+-8.81472,10.69285
+11.67857,10.44287
+9.38874,-10.40429
+-9.89273,-10.04607
+-10.81518,9.44361
+9.05229,8.13835
+10.45789,-10.42645
+-10.03634,-9.37133
+-11.07256,11.67951
+9.52707,9.53198
+10.63115,-11.36368
+-8.50256,-11.51555
+-9.32114,10.68318
+9.43624,10.69793
+10.73410,-10.45550
+-8.26749,-9.29010
+-11.78954,10.78791
+9.45826,10.59515
+10.99481,-10.10807
+-10.70363,-11.64026
+-11.43147,10.15081
+9.33839,9.46763
+9.26886,-9.88030
+-9.32112,-11.27544
+-10.62211,10.90514
+10.27965,10.17551
+11.13820,-11.31304
+-8.76586,-9.96605
+-11.10873,9.32534
+10.42517,8.81443
+9.50841,-10.10026
+-10.62705,-10.16282
+-7.99454,9.96844
+9.96749,8.74696
+10.38928,-9.79119
+-11.02115,-8.28531
+-8.85693,7.93110
+11.23170,9.44742
+8.80247,-8.41655
+-10.23274,-10.21180
+-12.57031,9.87308
+10.84163,11.05672
+9.54019,-7.91696
+-10.78474,-9.88344
+-8.45110,10.73121
+8.66103,10.07696
+10.26701,-10.91889
+-8.86627,-10.01438
+-7.41790,12.77705
+10.51506,10.36812
+10.21018,-10.73191
+-11.79217,-10.19714
+-9.47766,8.16040
+9.46847,9.92884
+9.24862,-7.99745
+-10.88589,-10.57881
+-10.52849,9.04576
+9.72932,9.69588
+10.43637,-8.03666
+-11.30674,-10.20216
+-9.37710,10.41503
+9.79516,8.73141
+9.61898,-10.88430
+-10.30770,-9.09768
+-9.30263,9.98449
+10.32470,8.88315
+9.92309,-9.84892
+-11.39970,-8.05178
+-8.09424,9.48515
+9.68576,10.00725
+11.70684,-8.31824
+-10.13734,-9.94913
+-9.53829,10.52766
+9.30354,10.31968
+10.60341,-12.64288
+-10.55030,-9.44511
+-9.56061,11.47782
+9.76510,9.37797
+10.08581,-8.80356
+-11.43452,-8.10385
+-10.11209,9.52933
+10.45915,9.38521
+11.06735,-8.80806
+-11.39254,-9.86397
+-8.90086,8.88200
+10.60686,9.42450
+10.09028,-9.71803
+-9.91630,-8.72941
+-9.60384,8.61556
+10.31291,9.50212
+10.87626,-8.65087
+-9.89494,-8.42328
+-11.82866,9.80568
+10.61253,10.81536
+8.65131,-8.88673
+-11.61610,-12.18677
+-9.19196,10.69086
+10.31952,10.48026
+8.85976,-10.82385
+-9.76009,-9.41045
+-8.79466,10.46762
+10.05094,9.24270
+9.00614,-10.67954
+-10.70536,-9.37649
+-9.50807,11.00805
+9.26111,12.14187
+8.40234,-10.66040
+-9.36061,-11.61721
+-9.93551,11.78598
+13.26857,10.52907
+10.14514,-7.58736
+-10.77337,-10.36379
+-9.64981,12.86875
+10.53872,10.37412
+8.88210,-10.42587
+-10.52182,-9.27849
+-9.96671,9.44245
+10.73093,8.44496
+9.28411,-10.45104
+-10.88354,-8.71007
+-9.59251,8.92294
+12.68011,9.54018
+10.48522,-10.74165
+-10.31666,-10.19553
+-8.74548,8.56810
+10.62848,9.57390
+9.27605,-9.10974
+-9.49841,-11.27113
+-10.84416,10.81291
+11.31406,9.39349
+9.90848,-10.42206
+-9.60380,-10.37283
+-8.99310,9.03374
+9.53842,9.42644
+9.26452,-8.72007
+-10.72715,-7.61051
+-8.55653,8.57659
+10.35199,9.11418
+9.98050,-9.06081
+-12.01775,-10.99910
+-9.80390,10.34856
+10.94779,10.19899
+9.03020,-11.75240
+-11.43345,-8.30540
+-9.33576,8.96582
+9.91431,8.55886
+11.24302,-10.26264
+-9.89263,-9.18396
+-10.64555,9.68114
+10.91600,9.51369
+11.40844,-9.39477
+-9.43976,-9.80090
+-10.48564,9.04878
+9.39284,8.50481
+10.06868,-10.29488
+-11.38176,-8.59655
+-8.15348,10.60448
+8.75879,11.40880
+9.11598,-9.09490
+-10.93990,-8.78800
+-10.58313,9.99642
+10.78348,8.06406
+10.29002,-10.30774
+-10.75515,-10.33659
+-10.98205,9.12853
+9.89118,11.44741
+8.45334,-9.68617
+-9.39782,-10.12327
+-9.71503,10.40295
+9.14251,11.45557
+12.46782,-8.91100
+-10.37926,-11.31158
+-9.94726,11.46988
+10.73491,11.34324
+10.31940,-7.88103
+-9.99199,-9.92971
+-10.16064,9.83339
+11.01309,12.07619
+10.15911,-11.14173
+-9.90406,-9.65842
+-9.92784,11.41626
+10.31270,9.44792
+11.61720,-10.57882
+-10.57321,-8.11685
+-8.15571,11.33389
+11.62870,8.65826
+11.03808,-9.02368
+-11.13726,-10.62132
+-8.89828,11.63450
+8.33481,9.09089
+10.72762,-10.26678
+-9.72395,-9.78895
+-10.08663,11.56466
+10.46232,10.51309
+9.79151,-10.56120
+-9.99365,-9.19203
+-9.80446,10.43761
+10.95559,9.72044
+12.47050,-10.57345
+-10.37743,-10.53196
+-9.90622,9.37593
+9.00454,9.94873
+9.25030,-10.14317
+-10.67162,-10.81071
+-10.46791,9.43856
+11.17755,9.56832
+9.05368,-10.37144
+-10.27444,-10.50792
+-8.95893,10.54339
+11.06394,11.05148
+9.43677,-12.96139
+-10.35745,-8.48170
+-10.74880,10.33331
+9.03730,10.50132
+11.41633,-10.57994
+-11.57394,-12.40522
+-8.38331,9.79896
+8.49386,10.09598
+10.58049,-10.15042
+-11.63347,-9.97697
+-11.25325,10.43794
+10.30143,10.05196
+9.24922,-8.88782
+-8.86199,-9.89980
+-12.14340,8.67006
+10.67871,10.45445
+9.14254,-10.53675
+-10.79841,-11.37195
+-10.28957,10.26206
+11.57349,10.47174
+9.10017,-10.60462
+-10.03015,-9.76499
+-8.78142,10.14470
+12.48554,9.98326
+10.54094,-10.73207
+-8.57703,-11.37881
+-9.85473,10.94727
+9.53259,9.89028
+10.78800,-9.59259
+-10.72480,-9.83470
+-10.59351,9.53960
+8.40097,10.37577
+10.42541,-10.78326
+-9.82957,-10.31652
+-9.88570,10.57116
+12.38786,9.73736
+10.67714,-9.75598
+-10.02757,-10.78444
+-11.12609,9.59362
+9.05743,9.88332
+11.46458,-9.61260
+-11.50079,-9.58956
+-8.76554,11.09407
+8.96338,9.28169
+10.49482,-10.04290
+-11.30623,-10.01282
+-10.82054,9.59366
+8.59035,9.99760
+9.49726,-10.93297
+-9.77937,-9.57368
+-11.80553,11.38116
+9.82366,10.27100
+9.85679,-10.98230
+-10.17774,-9.59803
+-11.08747,9.44812
+8.87199,7.49781
+9.82233,-9.82445
+-10.66175,-9.95100
+-10.46807,10.16945
+11.16649,9.75587
+11.21150,-10.33356
+-8.73045,-10.17117
+-10.24393,8.83691
+8.52754,10.52476
+10.66491,-9.64949
+-10.86709,-10.74813
+-11.39129,9.54333
+9.59596,10.26366
+10.53861,-7.19045
+-11.09406,-9.16796
+-11.43186,10.29037
+9.91290,8.25395
+9.81463,-9.83369
+-9.78981,-10.20711
+-10.30008,9.67528
+9.26711,9.92274
+8.79737,-11.08970
+-9.79957,-9.73644
+-8.63348,10.51728
+10.76833,11.11305
+8.11307,-11.46500
+-10.27220,-8.06495
+-12.14785,11.02530
+9.34343,9.88185
+9.02290,-10.60119
+-7.99165,-9.02355
+-8.56156,10.00446
+9.26728,9.90022
+9.79830,-8.81739
+-10.58173,-7.87813
+-9.59142,10.28573
+8.12315,9.46725
+11.07745,-9.70735
+-10.68059,-8.98704
+-9.78551,9.94560
+9.03648,10.50802
+10.35048,-7.78883
+-9.53755,-9.63709
+-10.60184,10.23686
+9.45788,10.67153
+7.70751,-8.66243
+-11.00667,-8.34721
+-10.23755,11.22358
+10.65623,11.04700
+10.65849,-10.03824
+-9.18513,-9.69681
+-8.88586,9.62882
+10.11888,9.09306
+10.46399,-8.32051
+-10.30704,-9.82585
+-9.47990,9.26605
+11.65922,11.30567
+8.81521,-10.57902
+-8.94680,-10.84854
+-9.28288,8.22583
+10.06906,11.16366
+9.40492,-9.79307
+-9.18090,-10.71693
+-9.80968,11.53057
+9.60009,8.36200
+10.56485,-11.34296
+-9.06538,-10.93051
+-9.27193,8.63900
+9.59191,11.09228
+9.60653,-6.03183
+-9.13497,-9.73679
+-9.47407,10.70571
+10.84349,10.03182
+10.50855,-8.12193
+-9.67922,-9.59827
+-7.73205,10.46383
+8.31109,10.56636
+9.28123,-9.39898
+-10.33461,-7.94511
+-10.84741,10.66485
+9.35674,9.41987
+8.72373,-10.79327
+-8.90546,-10.91235
+-8.92557,12.31973
+10.63585,9.88025
+10.43298,-9.32455
+-10.40159,-11.92945
+-9.20847,11.53923
+9.14252,9.63573
+10.61093,-9.58711
+-10.30117,-9.88536
+-10.58194,10.21532
+11.58790,9.88059
+8.32446,-8.20048
+-10.20161,-9.61202
+-11.37331,10.89353
+10.59982,7.92503
+9.07854,-9.35921
+-11.26928,-10.80880
+-9.90672,9.05920
+8.21218,9.91095
+10.59023,-10.44667
+-10.33640,-11.64252
+-10.64815,10.70069
+11.27640,10.36133
+11.30897,-9.45895
+-10.16719,-10.73363
+-11.23383,9.52494
+10.43839,9.82666
+10.85455,-9.10510
+-10.80915,-10.75267
+-12.37611,9.60041
+10.83143,10.68938
+10.45843,-9.82701
+-9.62727,-10.98081
+-10.92198,10.00248
+9.66418,8.68171
+10.15973,-10.61065
+-10.93813,-10.76756
+-10.18143,10.12596
+10.23370,10.86373
+10.72737,-9.61228
+-10.11652,-8.92969
+-11.48778,11.73070
+9.04521,9.27980
+9.43621,-10.63351
+-8.39304,-11.80930
+-11.14232,10.13091
+8.32057,10.53107
+11.09140,-8.91163
+-10.61629,-10.47461
+-9.98486,9.74578
+9.93790,10.79100
+11.13299,-10.85914
+-11.03955,-11.18250
+-9.95265,8.99175
+9.14249,10.00653
+9.74654,-10.80388
+-9.44367,-10.50675
+-9.26350,9.53834
+11.24345,11.22143
+10.03167,-9.47031
+-7.99487,-10.11138
+-8.48946,9.30034
+8.71232,10.74311
+9.37262,-9.62510
+-10.90845,-9.70845
+-10.72285,11.12848
+9.14538,10.66610
+9.15607,-11.80157
+-9.40604,-9.76287
+-9.06436,8.68669
+9.27082,9.62454
+11.36449,-10.50794
+-9.62961,-8.05942
+-10.11833,8.93896
+7.93840,9.80334
+10.38358,-11.34545
+-8.97645,-9.33673
+-9.50367,11.15025
+8.06955,8.99552
+10.87361,-10.31623
+-9.53984,-10.53960
+-11.36110,11.07691
+8.03776,9.47330
+10.16859,-10.16159
+-10.97693,-10.35720
+-10.32402,9.96309
+10.10589,9.06415
+10.28208,-9.60426
+-9.69976,-9.84454
+-9.58335,10.38769
+9.90505,8.15701
+11.93997,-9.84351
+-10.04465,-10.41275
+-9.48266,9.83035
+11.01960,9.74856
+9.80910,-9.07617
+-10.43369,-9.57526
+-10.36310,10.90975
+10.89667,11.08836
+10.45869,-11.88607
+-10.97066,-8.76100
+-9.41593,10.37913
+9.63651,9.74417
+9.70560,-11.30898
+-10.53688,-8.08429
+-10.84107,9.24640
+10.25541,10.78392
+10.02273,-9.49320
+-8.17242,-10.64942
+-9.71593,10.26981
+9.08211,10.73400
+9.05968,-8.04733
+-10.67189,-8.99162
+-10.08695,9.00906
+10.59805,10.21295
+9.20821,-9.97307
+-10.11423,-10.25986
+-9.51026,9.39571
+10.92290,8.79973
+9.81790,-9.12368
+-11.08377,-9.73456
+-11.92440,10.47324
+9.92516,9.87340
+10.31059,-10.43031
+-8.71257,-9.98315
+-10.72789,10.08214
+10.24777,10.51931
+9.98164,-10.52744
+-8.58714,-10.26385
+-9.08275,10.05367
+10.36835,10.20427
+7.94811,-10.81784
+-10.31926,-10.40704
+-10.57235,9.82277
+11.14358,9.65682
+10.36067,-8.64399
+-9.10832,-10.50254
+-9.42650,10.60239
+11.80894,9.57362
+11.73009,-10.57912
+-8.70715,-9.75159
+-10.94680,11.49147
+9.32617,10.94583
+10.61940,-10.70066
+-9.64511,-10.93981
+-8.95545,10.01221
+11.31149,11.76301
+11.21711,-11.27727
+-9.41775,-9.85545
+-8.63477,10.38334
+9.68066,9.57169
+10.45486,-10.39564
+-10.16133,-11.34478
+-10.23672,10.28395
+11.40109,10.20316
+10.35643,-9.95919
+-8.82024,-10.21282
+-10.59611,10.64521
+10.35039,10.19665
+10.47411,-7.34364
+-10.74941,-9.20666
+-10.29022,10.55151
+9.46901,9.97715
+8.15745,-9.34204
+-10.66975,-10.83091
+-9.73891,9.74562
+9.78628,12.06999
+9.85998,-10.08235
+-9.10009,-8.45769
+-10.18325,9.04937
+11.23699,10.37527
+10.30426,-9.32250
+-8.47053,-9.65377
+-8.68369,10.52153
+11.00470,11.27149
+10.24327,-10.62426
+-9.92842,-10.77604
+-9.57049,7.64888
+10.56260,10.40175
+10.63876,-9.64787
+-10.59516,-11.65764
+-11.48889,11.06554
+11.00955,10.06101
+10.02032,-9.66686
+-9.27787,-10.39721
+-10.73128,9.34136
+11.44648,9.70714
+10.51658,-10.73376
+-8.92928,-10.47320
+-9.40933,9.96875
+7.81226,10.99374
+12.52023,-10.53572
+-9.83602,-9.87525
+-9.71818,10.23798
+10.42220,11.07626
+11.42910,-9.50515
+-10.87067,-9.55424
+-10.47834,9.44444
+10.36893,9.51347
+10.81912,-11.67926
+-11.69170,-11.08815
+-8.62041,11.04617
+10.73926,10.77789
+9.78918,-9.38079
+-9.82235,-7.48713
+-10.49710,10.77552
+11.23765,9.16493
+9.22478,-9.28142
+-9.51236,-9.92319
+-10.10974,11.20404
+9.06409,8.19344
+10.49498,-10.13777
+-10.97822,-8.74542
+-10.75200,10.40608
+9.40911,10.98030
+9.03404,-8.22448
+-11.29419,-10.56329
+-9.51800,9.43113
+10.02925,9.85512
+9.62837,-9.57323
+-11.43336,-9.59163
+-9.98509,8.23803
+11.39496,10.50071
+11.14867,-9.84804
+-8.47876,-8.58353
+-9.84511,9.31082
+9.11190,9.04252
+9.76777,-9.67472
+-10.42655,-8.39439
+-10.79421,9.83319
+9.55596,9.88564
+10.25675,-9.19885
+-10.77327,-9.97272
+-9.55755,9.56761
+10.85290,9.92819
+9.80119,-11.61077
+-9.20456,-9.54198
+-11.13315,11.23772
+10.77529,10.07254
+9.01900,-10.44040
+-10.17269,-10.42030
+-9.90074,10.80687
+9.22738,8.78598
+10.62694,-10.68046
+-10.21464,-10.69159
+-9.31182,9.72341
+7.82974,10.93901
+11.73415,-10.30120
+-9.19740,-8.99956
+-9.32926,9.76386
+9.37870,10.22651
+10.31677,-9.88284
+-11.67609,-10.16991
+-10.17450,11.56157
+9.67360,9.67415
+9.62287,-9.58162
+-8.85967,-10.92191
+-11.23588,10.93743
+11.12515,9.22862
+10.83520,-7.72434
+-9.40191,-10.34580
+-10.12823,10.42448
+9.07179,9.94202
+9.93700,-10.14004
+-9.35474,-7.86098
+-10.10071,11.31228
+11.14632,8.16719
+9.17037,-8.24084
+-10.61005,-10.19286
+-8.42737,9.57242
+10.63423,8.00922
+10.47292,-9.18983
+-7.85406,-8.89202
+-10.42172,9.73758
+11.45558,9.22816
+10.77728,-9.36780
+-8.98101,-10.69173
+-10.51112,10.09743
+9.15765,9.93097
+7.92523,-11.93466
+-9.32817,-9.22466
+-11.18977,9.74448
+9.11362,9.16552
+10.81267,-9.97035
+-9.41767,-8.61645
+-10.13574,10.87007
+9.56766,10.03826
+9.69717,-10.97689
+-10.93551,-11.01146
+-10.55517,10.10656
+9.61906,10.23607
+10.23658,-9.70687
+-9.95956,-9.37172
+-9.18215,9.33752
+10.18449,8.75968
+10.83192,-9.12924
+-11.97817,-11.82667
+-10.34487,10.41948
+9.01702,10.52806
+10.10012,-12.09842
+-8.58187,-10.01818
+-11.10289,9.34528
+11.23304,9.45171
+9.06245,-10.49049
+-8.20648,-8.34392
+-11.24282,8.56408
+9.90488,12.32164
+12.07694,-7.98433
+-12.21207,-9.33215
+-10.13118,11.13652
+10.45890,9.00335
+10.06195,-8.28762
+-11.04265,-9.12531
+-10.76484,10.81081
+9.40921,10.08980
+9.55566,-8.89182
+-9.60624,-11.09752
+-9.78502,11.61252
+10.97856,8.40465
+10.54652,-8.72873
+-10.37030,-8.75363
+-8.94254,9.93310
+8.83600,11.23634
+9.72048,-10.65667
+-9.63377,-10.49152
+-11.20997,8.82166
+11.20461,9.54338
+10.78393,-10.15550
+-10.09594,-11.06405
+-9.45304,10.59606
+10.63346,10.40061
+7.97153,-12.25659
+-9.62405,-9.11866
+-9.90586,8.32722
+10.53623,8.76538
+11.51528,-10.24515
+-10.33156,-7.91434
+-8.92471,11.01810
+9.15019,9.45540
+10.09384,-8.60936
+-9.28634,-8.96253
+-9.36963,9.68765
+10.47758,10.52279
+10.90339,-10.98987
+-8.75271,-11.36841
+-10.32607,9.87859
+9.43273,10.37153
+11.08157,-9.05475
+-8.10743,-8.78851
+-6.95538,9.88903
+10.34070,10.19543
+8.01213,-8.00066
+-10.90769,-11.44841
+-11.44141,10.03386
+7.72780,10.57333
+8.12598,-10.47668
+-11.90671,-10.97679
+-10.94069,10.09971
+11.12259,9.52380
+9.96840,-11.95725
+-9.25447,-11.12460
+-11.31256,8.11973
+8.83465,10.86097
+10.59261,-10.15478
+-10.70842,-8.39674
+-9.84836,10.79335
+8.63837,10.76365
+11.39025,-11.33752
+-8.09869,-9.39632
+-11.04070,8.78429
+9.80315,11.63859
+9.43823,-11.06204
+-7.86063,-9.76461
+-9.40110,10.25994
+11.59261,10.16369
+8.72797,-9.22981
+-9.47783,-9.78010
+-8.50480,10.63264
+12.15902,11.04966
+9.52550,-10.72505
+-12.79116,-9.62224
+-9.65679,7.65986
+9.23804,11.29339
+12.24501,-8.51803
+-9.47643,-10.18162
+-9.48579,10.55746
+11.72191,8.10246
+8.93076,-9.40097
+-8.16331,-10.46997
+-9.85006,8.28852
+9.59827,8.81361
+9.52710,-11.28429
+-10.41406,-9.60134
+-10.09253,9.90311
+11.56178,10.33068
+8.97024,-9.07942
+-11.98459,-8.96083
+-10.86499,10.38770
+11.07976,9.78610
+9.69450,-11.78051
+-10.39235,-9.35607
+-8.53224,11.03293
+8.85815,11.07613
+11.04226,-11.32937
+-11.71920,-7.71100
+-11.48612,10.23680
+8.96093,11.54670
+9.49699,-8.73766
+-10.31412,-8.83411
+-9.92698,11.53172
+10.79845,9.02498
+9.81433,-8.31909
+-10.29297,-9.66703
+-8.97640,8.86330
+10.85751,10.48841
+11.11436,-9.94196
+-10.87961,-8.72874
+-10.24447,9.36091
+9.41348,7.31984
+9.79543,-9.03830
+-11.76415,-10.93578
+-11.23427,11.30015
+10.12746,9.65638
+10.68613,-10.69257
+-11.03585,-9.78550
+-10.91629,7.21007
+10.63165,9.70225
+11.74147,-9.08562
+-9.96620,-9.92746
+-9.83225,11.96545
+10.46393,9.87808
+9.32593,-10.24950
+-8.85635,-8.36512
+-10.13375,10.29366
+9.84185,11.43196
+10.13281,-10.95740
+-10.34673,-9.42887
+-9.08977,10.66029
+9.27005,9.64722
+12.16829,-9.39180
+-11.14936,-8.50696
+-11.18506,12.02585
+10.26447,10.34396
+9.71484,-10.35664
+-9.21518,-8.49607
+-10.42279,8.98880
+11.41020,9.68538
+9.54841,-10.35655
+-10.05680,-9.20058
+-8.13889,8.91186
+10.59435,10.00340
+9.86053,-8.75082
+-11.05684,-10.18061
+-9.58728,9.51231
+9.82294,9.65159
+9.49603,-12.07346
+-9.41327,-9.88477
+-9.08882,9.87624
+10.26225,12.53301
+8.64548,-9.28806
+-8.80531,-9.99606
+-10.42859,11.76859
+9.22098,8.30358
+9.20210,-11.53770
+-11.15363,-8.75577
+-9.46846,10.41907
+10.30272,9.88234
+10.32384,-10.15257
+-9.27416,-9.27911
+-9.00806,11.50839
+10.26830,10.70530
+11.45553,-10.19589
+-10.19607,-10.75932
+-11.14292,10.75641
+11.04880,9.39039
+9.01844,-10.73886
+-10.20580,-10.29995
+-10.08369,9.56955
+8.20153,9.45054
+9.64818,-9.20783
+-10.00917,-9.17049
+-9.26841,9.93291
+11.56447,10.27208
+9.32410,-9.66530
+-9.28004,-11.57094
+-12.00939,9.06174
+9.20961,11.55796
+9.91133,-10.22537
+-9.66669,-10.58589
+-9.32737,8.65596
+10.34644,8.49550
+10.72106,-10.81015
+-9.07737,-8.47644
+-8.94564,10.91834
+10.20607,9.20829
+10.90056,-10.96402
+-11.09950,-10.63436
+-9.65849,11.44054
+10.09714,9.39858
+8.69323,-9.42362
+-9.48381,-9.92220
+-9.98319,10.24475
+12.23359,11.15158
+10.60198,-10.13227
+-12.21439,-10.19619
+-10.33209,9.24901
+9.73020,9.96656
+11.36550,-10.08367
+-8.72619,-10.14161
+-9.51303,10.19468
+10.35897,10.49702
+10.55125,-9.21873
+-9.06086,-11.02142
+-10.14363,9.36323
+10.09069,10.00661
+11.27784,-8.71646
+-10.74858,-10.66108
+-9.72767,9.92491
+9.80953,11.78833
+9.79111,-10.15729
+-11.69363,-10.66828
+-9.97807,9.93690
+9.29402,9.67025
+9.29956,-8.46169
+-10.09699,-9.80941
+-9.11142,9.45665
+10.60901,10.81594
+8.74685,-9.97517
+-9.38810,-10.11134
+-10.52933,10.01762
+8.80826,8.78512
+9.79414,-10.06593
+-10.95086,-10.13946
+-9.34755,11.85037
+10.33690,8.71539
+10.25068,-8.92740
+-10.22454,-10.29497
+-9.86132,11.72122
+9.89474,11.02733
+10.49675,-10.57387
+-10.26880,-9.64483
+-10.13370,9.82448
+9.80235,8.59695
+13.64861,-10.00954
+-10.26126,-10.50298
+-10.36640,10.30377
+11.88544,9.02442
+8.80548,-11.04484
+-9.93435,-10.48625
+-9.13812,10.43591
+10.88628,8.89601
+11.23828,-11.22852
+-8.32250,-10.37900
+-8.94260,10.57560
+10.89314,10.07769
+8.10701,-9.49700
+-8.80041,-8.29578
+-9.03579,10.91354
+10.60621,9.28149
+9.30157,-10.32451
+-9.43958,-9.99516
+-10.72909,8.95833
+9.40143,9.80661
+9.81275,-11.45257
+-8.65311,-9.94100
+-9.33147,12.61211
+9.27312,9.12073
+8.75619,-11.45196
+-10.32631,-9.91007
+-11.70754,10.62705
+9.50012,10.80845
+11.03241,-8.62137
+-10.62979,-11.15903
+-9.56262,8.46199
+7.66580,10.07044
+10.66475,-10.84371
+-11.27166,-11.37696
+-10.97934,9.00836
+10.12265,10.87440
+11.17675,-9.82832
+-10.38451,-11.48352
+-8.37816,11.37049
+10.53241,9.03431
+10.78666,-12.27342
+-11.37937,-9.97841
+-9.94381,9.48782
+10.09785,10.01023
+8.68097,-9.39509
+-10.21281,-7.96522
+-10.52730,9.58776
+9.95937,10.76236
+10.28237,-9.49581
+-11.98465,-9.77173
+-9.46421,9.55579
+9.93557,10.28646
+10.24551,-8.90940
+-8.95981,-10.46152
+-11.37831,9.66163
+8.82101,10.53811
+8.95769,-9.36666
+-10.85955,-8.54623
+-9.43505,9.53250
+9.66455,8.60149
+9.43738,-10.87827
+-8.40623,-8.36922
+-9.36913,10.70491
+10.30498,11.22257
+11.03261,-10.57133
+-9.74972,-9.73540
+-10.48271,11.76154
+8.32755,10.85854
+11.30540,-9.97884
+-10.02942,-10.96490
+-10.57989,11.24509
+8.28117,9.89107
+8.03966,-9.44623
+-8.44677,-11.66668
+-8.75400,10.96751
+9.76473,8.98625
+9.06730,-10.63488
+-9.90956,-10.32810
+-8.57582,10.19621
+11.64864,8.56020
+10.05381,-9.99620
+-10.38008,-9.85134
+-9.56266,11.80383
+9.64731,9.79651
+9.47729,-10.62084
+-9.48229,-10.64971
+-10.10103,9.77926
+7.80542,9.86898
+10.06796,-10.13635
+-9.62061,-9.22922
+-11.90561,10.86249
+10.47356,9.85067
+9.57358,-9.68392
+-10.84707,-8.96885
+-11.12753,13.03356
+10.37216,9.35840
+11.13916,-9.78911
+-9.47874,-9.37676
+-9.98124,10.79307
+9.27339,11.21850
+9.51716,-8.50290
+-12.17339,-11.94945
+-9.05211,9.51754
+11.59195,9.60478
+10.58029,-8.42138
+-10.31632,-9.54258
+-9.89842,11.22793
+11.02405,10.68130
+10.53029,-12.03508
+-9.63620,-9.49423
+-10.27580,7.66573
+9.62426,11.17516
+9.88930,-9.52432
+-10.19773,-11.82596
+-9.71754,10.90756
+10.75540,10.90447
+9.19644,-9.84722
+-10.02477,-9.56135
+-10.81598,11.02695
+9.75996,8.55604
+10.98841,-9.19549
+-10.41219,-10.48228
+-9.20602,10.31217
+7.39166,8.27359
+8.69900,-11.21874
+-8.63282,-8.62120
+-9.21391,10.14803
+11.22442,9.32812
+9.29512,-10.62999
+-11.30131,-10.10223
+-8.03921,9.07136
+12.14252,10.66823
+11.56786,-11.34368
+-10.57101,-9.79634
+-10.48260,10.79437
+11.30112,9.63441
+8.72573,-8.90159
+-9.47794,-10.74279
+-10.17448,9.31024
+9.49678,9.18044
+8.79943,-10.82590
+-10.06826,-10.94577
+-10.83430,9.28380
+9.25704,9.95402
+9.93389,-10.84095
+-11.38570,-9.36605
+-8.12311,9.25645
+8.61426,8.91614
+9.19004,-10.00996
+-10.29770,-9.04394
+-9.56922,7.94011
+10.24399,9.86606
+10.20059,-10.29932
+-8.44613,-10.49425
+-8.35630,10.16313
+10.65330,8.88826
+10.80380,-11.03928
+-9.90875,-9.06502
+-10.10991,9.53656
+9.61221,12.72476
+10.44392,-10.91831
+-8.85923,-8.56117
+-10.32139,10.42122
+10.69221,9.79368
+7.62567,-9.19731
+-8.65944,-12.67489
+-10.60873,7.51352
+9.36386,9.89522
+11.12045,-9.75029
+-10.54497,-9.96116
+-9.54277,9.15576
+9.75832,10.09670
+10.06605,-8.47953
+-11.42397,-10.55341
+-9.61040,9.70589
+8.71674,9.43861
+10.30281,-8.94191
+-7.84210,-11.75214
+-10.24161,7.86337
+9.82101,9.79327
+10.44666,-10.74912
+-8.82361,-10.05077
+-10.24698,10.05227
+9.71565,9.51185
+11.46914,-11.30898
+-10.52750,-10.76636
+-9.78236,9.18586
+8.24318,10.15144
+8.58822,-10.28426
+-9.32508,-10.13643
+-9.86549,9.05159
+8.98240,9.59490
+9.72403,-11.90485
+-9.81094,-8.58320
+-9.30800,11.21934
+10.29368,9.43534
+9.59161,-8.77023
+-9.37510,-10.82530
+-10.06084,9.53003
+10.10544,9.57301
+10.18298,-9.83365
+-11.50262,-12.06065
+-10.70105,9.69475
+9.76329,8.52281
+10.84961,-9.50332
+-11.05083,-9.53927
+-8.64393,8.59392
+8.79818,10.52039
+8.99822,-10.73121
+-9.00184,-11.65049
+-10.92139,8.99964
+9.38743,9.87826
+12.30170,-11.32935
+-9.87494,-10.57364
+-10.38747,9.99393
+9.89774,12.07664
+8.72776,-10.46614
+-10.25042,-11.16734
+-8.74579,9.36417
+9.64452,9.99896
+8.58582,-7.34529
+-8.73182,-10.24439
+-11.76997,9.71413
+9.69048,9.72602
+8.81813,-10.87897
+-9.49897,-7.88605
+-9.97436,11.70032
+11.60749,10.37488
+11.18561,-11.18474
+-9.06484,-9.78741
+-9.22133,11.41888
+10.30471,9.20098
+10.20096,-10.11758
+-8.83265,-8.03437
+-9.63656,10.87100
+9.82643,8.84976
+10.21152,-11.28848
+-10.72168,-10.85220
+-10.49610,12.52120
+9.02762,10.39170
+9.63073,-11.90887
+-11.57507,-8.91945
+-6.87278,10.07388
+9.15182,8.07617
+8.05157,-10.68256
+-9.33012,-7.64969
+-11.35974,10.63981
+10.77655,8.71432
+10.74401,-9.36099
+-10.16125,-10.02034
+-9.39571,10.11879
+9.89176,9.87196
+10.39557,-10.49633
+-11.80855,-11.34506
+-9.50602,9.02743
+11.68777,9.64452
+10.47443,-11.14409
+-9.49697,-11.99801
+-9.56243,9.81657
+11.31917,9.38214
+11.29860,-10.13238
+-10.60543,-9.62481
+-10.73386,9.83689
+9.98778,10.32454
+9.87984,-10.08902
+-9.74297,-11.03194
+-9.12235,8.88741
+10.31995,11.25135
+9.58506,-10.01128
+-9.55274,-8.76771
+-9.73403,9.64266
+9.66181,10.83563
+10.45640,-9.17509
+-10.53374,-9.66689
+-10.82855,11.68729
+11.33044,7.52024
+10.12468,-10.13461
+-9.43930,-11.23664
+-8.30105,10.61634
+9.54490,11.96652
+11.00821,-9.71036
+-8.73478,-7.92500
+-8.55205,9.01528
+12.28558,10.73249
+9.49618,-10.67041
+-9.14819,-8.73255
+-10.83634,11.15734
+11.17886,11.03595
+9.86144,-9.76321
+-9.81032,-10.06191
+-10.86520,8.56682
+8.54088,8.08112
+8.87362,-10.63992
+-8.29748,-8.80081
+-11.08836,10.50831
+9.11140,9.29561
+9.20531,-10.28005
+-10.97931,-10.95176
+-10.64196,10.33425
+8.80794,9.52256
+10.06391,-9.61654
+-9.26943,-11.50136
+-11.35627,9.91730
+11.31454,8.79609
+10.92434,-10.16882
+-8.66068,-8.78295
+-10.81800,9.87393
+8.58158,9.81058
+11.40775,-10.42354
+-9.83138,-8.97320
+-11.99374,10.57698
+10.30339,8.97821
+8.61619,-11.61281
+-10.71276,-9.28082
+-11.05641,8.60564
+9.22961,9.60208
+12.85201,-8.67674
+-10.31873,-11.30022
+-9.04246,10.03854
+9.93450,11.13977
+9.49304,-8.35232
+-9.78314,-8.24701
+-10.32338,10.84076
+11.29883,9.74692
+9.59317,-9.18067
+-9.60250,-9.39926
+-8.97849,11.42010
+10.04514,11.23236
+11.45640,-10.85587
+-8.61010,-9.46654
+-10.80089,8.09647
+8.56212,10.38245
+9.35573,-9.67965
+-9.92369,-11.20451
+-9.81577,8.69662
+9.09316,10.87759
+9.64602,-11.06260
+-9.27313,-10.16598
+-10.40464,11.52408
+9.85324,10.37337
+8.97033,-8.96011
+-10.97341,-10.20775
+-10.25952,9.53855
+9.72026,9.52617
+10.91760,-10.13090
+-11.35450,-11.17696
+-11.38985,11.19447
+10.60943,8.85197
+8.23198,-11.44350
+-12.52232,-9.17271
+-9.06091,11.53304
+9.58592,10.61441
+10.20667,-10.16407
+-10.06016,-8.51148
+-10.34189,10.40464
+10.39216,8.63393
+10.99262,-9.68107
+-11.35130,-10.80247
+-10.55082,9.40308
+12.22409,9.21635
+9.69951,-8.08435
+-11.11446,-9.48288
+-9.68895,10.44284
+10.96888,8.81415
+9.87599,-10.14014
+-11.34014,-10.89932
+-6.98483,10.76602
+10.98920,9.98488
+9.97595,-9.90268
+-10.02832,-9.24534
+-7.61760,9.62471
+9.60609,10.48619
+11.23470,-9.01899
+-10.12413,-10.48708
+-9.27441,11.51639
+11.01407,10.24986
+9.71485,-10.55926
+-10.11380,-11.78044
+-10.28152,8.98658
+9.66668,9.89833
+10.60628,-9.36398
+-10.42292,-8.74587
+-10.20737,9.98638
+10.28229,10.89040
+9.82663,-7.90773
+-12.26615,-8.78506
+-10.45290,10.06004
+9.16224,10.35202
+9.19272,-9.31449
+-10.54579,-11.04041
+-9.02675,9.94682
+8.94292,9.31063
+11.04800,-10.08688
+-11.00674,-8.65868
+-9.68876,10.06258
+11.90378,9.46336
+9.52665,-7.87738
+-10.90065,-11.09848
+-12.17516,9.39688
+9.17696,10.46965
+9.80210,-9.70711
+-8.53034,-9.53689
+-9.69752,10.23212
+9.97303,10.19398
+9.70187,-11.08042
+-8.44446,-11.46486
+-10.06174,12.11354
+10.40889,10.00437
+10.16634,-11.91674
+-10.08796,-8.54313
+-11.01530,10.70528
+9.77267,10.12061
+9.94896,-11.89702
+-9.58866,-8.53985
+-9.30677,9.39395
+10.04517,9.36781
+8.63740,-12.15326
+-9.03594,-9.29222
+-11.49872,11.62536
+10.95029,11.93029
+11.01518,-8.12818
+-10.10896,-9.37097
+-10.79577,10.64224
+11.69126,11.09083
+11.54676,-9.62817
+-8.47004,-9.08035
+-11.51503,8.90385
+11.06166,9.78698
+9.35803,-10.99709
+-9.71063,-10.41768
+-11.05770,8.63879
+9.68060,11.38224
+10.29201,-8.58574
+-9.44646,-10.68658
+-9.86014,9.51888
+9.15063,8.97429
+7.76888,-11.35674
+-8.89104,-11.19801
+-9.40219,10.29705
+10.37588,10.67547
+11.33661,-9.99008
+-10.06777,-9.37963
+-9.87390,13.26037
+9.94066,8.89928
+10.59117,-11.18708
+-8.75182,-9.29836
+-9.54861,11.18791
+10.51534,8.91911
+9.66520,-7.33836
+-10.15476,-9.79827
+-10.35463,9.41615
+10.96609,8.93402
+10.81571,-7.96566
+-9.64693,-9.80077
+-9.40480,8.75272
+9.78873,9.00485
+11.13513,-10.07626
+-9.15549,-11.54010
+-9.93475,10.35064
+8.76971,8.83107
+11.58374,-8.99492
+-9.59345,-10.03794
+-9.08575,8.94148
+10.02513,10.25190
+10.57879,-9.67782
+-9.60874,-10.80263
+-9.05621,11.52811
+11.03177,9.68811
+10.35635,-10.45514
+-9.11004,-9.02337
+-9.31948,8.75901
+10.62985,9.07006
+10.35493,-9.18981
+-10.45232,-9.84226
+-10.87112,9.32626
+11.96094,10.28453
+10.03302,-10.11520
+-10.49267,-8.93543
+-11.14206,7.36634
+11.70725,10.84637
+11.28232,-10.70400
+-8.51321,-10.68801
+-9.37059,10.46603
+9.60313,8.13683
+9.74379,-10.12884
+-11.63330,-9.34923
+-11.04468,9.80213
+10.88020,10.49250
+11.64909,-9.45907
+-9.82263,-9.79076
+-9.80111,10.61693
+11.42207,8.67481
+10.55587,-10.83234
+-9.30936,-9.12964
+-8.56297,9.54721
+11.30649,11.09037
+10.11536,-10.94704
+-9.93213,-8.76877
+-9.79485,11.10498
+11.53138,8.42556
+10.00841,-9.76871
+-11.40717,-10.16934
+-10.73078,12.42997
+9.49112,9.47356
+8.16886,-9.43908
+-9.58465,-10.29199
+-10.46081,9.64453
+10.44533,11.03420
+9.18080,-11.62502
+-8.96809,-9.04800
+-9.47067,8.81180
+10.70513,7.69558
+9.22083,-11.61611
+-9.73661,-9.44242
+-9.52585,10.20806
+9.14841,10.21663
+10.02041,-8.97340
+-9.60532,-9.46961
+-10.14239,11.04176
+8.88160,10.10075
+9.68069,-9.90912
+-12.18377,-9.74342
+-11.90152,10.75067
+9.33713,10.74420
+11.01766,-8.57933
+-8.40739,-9.46816
+-10.62317,8.50220
+9.17690,7.82677
+9.71879,-9.18973
+-10.82821,-10.85012
+-11.12101,9.91150
+9.40246,10.53356
+9.31760,-9.06082
+-11.20021,-9.97525
+-9.29860,10.40177
+9.66138,10.84923
+10.00743,-9.33317
+-9.94947,-9.76835
+-8.01807,11.43965
+9.15121,11.33423
+9.26526,-10.06274
+-11.01999,-10.01313
+-11.47395,9.21298
+10.02850,10.86330
+9.89555,-11.05218
+-9.82982,-8.74657
+-8.77474,8.98495
+8.90512,8.19925
+8.64409,-11.31966
+-9.95587,-10.03744
+-12.23260,8.17909
+9.20754,11.54143
+11.43634,-7.99381
+-8.24916,-10.02577
+-9.96426,10.43956
+11.55755,10.79276
+10.69139,-9.75332
+-10.00847,-8.61612
+-10.42279,9.51833
+10.48663,9.92037
+8.84087,-10.35644
+-10.16742,-9.49678
+-10.82848,10.65934
+9.73486,8.90579
+10.37038,-9.14261
+-8.69662,-11.13853
+-9.86009,9.44894
+8.87271,9.68390
+10.38820,-9.57865
+-9.94017,-9.86458
+-9.18865,10.01345
+9.81374,9.64153
+9.20952,-10.99777
+-10.34791,-9.82034
+-9.74684,9.61834
+8.41610,10.56778
+9.92427,-10.19012
+-10.03744,-11.15876
+-7.84506,9.10043
+9.96717,9.53874
+9.11871,-8.51254
+-8.20600,-10.99095
+-9.26489,8.18930
+10.63120,10.42960
+10.58889,-12.15059
+-9.14493,-9.17382
+-8.37845,8.93771
+11.00843,10.80629
+10.13734,-9.39641
+-9.81003,-10.31344
+-10.19067,9.78712
+10.80865,9.38547
+10.15855,-10.73741
+-9.02453,-12.46408
+-8.14014,10.69019
+10.07525,11.03643
+7.64898,-10.02127
+-10.42664,-9.81743
+-8.67317,10.26881
+10.00907,10.96956
+9.26995,-9.42124
+-7.83799,-10.68839
+-10.71673,10.51174
+9.49110,9.06149
+8.55857,-11.65178
+-9.09631,-10.50533
+-11.39685,9.43517
+10.77438,10.34321
+9.54243,-11.24502
+-9.59232,-8.26617
+-10.64206,10.71593
+10.41395,12.06740
+9.99002,-9.66592
+-9.14788,-8.19177
+-9.51716,10.20049
+9.91810,11.34135
+9.29910,-9.88684
+-7.70754,-8.23765
+-9.54038,10.80727
+10.93871,10.48593
+9.69661,-6.79168
+-10.99619,-10.25315
+-9.32692,9.03428
+11.83055,12.72580
+8.47919,-10.78704
+-9.96854,-8.38640
+-9.80022,10.72919
+10.17587,8.41966
+10.41697,-9.31876
+-8.59010,-9.72549
+-9.37188,9.28207
+9.69794,12.59276
+9.60728,-10.55404
+-10.48388,-10.18353
+-9.89797,10.76094
+9.17629,11.51942
+9.87541,-10.61566
+-9.52021,-9.69515
+-9.47688,12.77982
+9.46323,11.09295
+11.42765,-7.53693
+-9.71872,-9.63185
+-8.82274,10.06291
+11.40252,11.41009
+8.89692,-9.93163
+-11.66551,-10.32089
+-8.71208,11.49049
+11.85056,10.46496
+11.49416,-9.42032
+-10.99003,-11.16397
+-9.22391,9.27810
+7.52565,9.38779
+9.81864,-10.49501
+-9.09910,-11.95268
+-8.12370,9.22936
+10.35243,11.24678
+10.16178,-9.40952
+-10.39705,-10.48135
+-9.82518,8.68557
+8.94945,10.10439
+12.53047,-11.09066
+-9.36598,-10.09295
+-10.08774,8.71499
+11.94359,10.17305
+10.26652,-8.87170
+-10.99833,-11.78310
+-8.65501,9.74306
+11.81121,10.59936
+8.50454,-10.86670
+-9.71439,-8.82713
+-10.22721,8.53695
+9.47220,10.88962
+10.89106,-9.76999
+-9.81664,-10.14791
+-10.48501,11.79327
+11.09183,12.43557
+10.17270,-10.32268
+-8.43109,-9.57235
+-9.34072,10.68726
+10.36224,8.71872
+9.94699,-8.76244
+-8.06958,-10.91905
+-8.95684,8.69496
+9.30940,8.69560
+10.99460,-8.57467
+-8.82019,-9.30171
+-9.80341,10.29698
+9.68654,9.16742
+11.20154,-10.54502
+-8.86759,-7.70880
+-9.83048,8.64628
+7.51513,10.29012
+10.06868,-9.41459
+-9.98532,-10.30897
+-9.55517,9.81280
+9.63769,10.09262
+11.92751,-8.59546
+-10.33195,-10.69933
+-11.59253,10.44339
+9.63561,10.12190
+8.83745,-8.44008
+-10.51152,-8.83333
+-11.45937,10.80352
+9.17154,9.99931
+11.62780,-10.76219
+-11.06099,-10.08239
+-9.86229,9.44323
+10.25358,8.95589
+8.85924,-12.12081
+-9.78984,-10.08318
+-11.38842,10.23454
+11.25666,9.51026
+11.38471,-10.22370
+-12.47890,-11.64407
+-10.03735,9.92935
+8.36482,10.05217
+9.65034,-9.58218
+-10.85787,-9.24916
+-11.76254,8.40765
+9.47801,9.26694
+10.81049,-9.58735
+-9.46129,-10.03316
+-9.77420,8.63404
+11.20857,9.79298
+7.55426,-11.18165
+-10.05145,-7.87934
+-10.94737,10.52859
+10.64923,7.96477
+10.27346,-11.46262
+-10.60757,-10.65065
+-9.37005,9.72513
+9.49135,9.86700
+11.34872,-9.47376
+-9.50499,-10.03895
+-10.30865,11.35197
+11.74025,10.01916
+8.75132,-10.56152
+-9.81403,-10.65209
+-9.33923,9.27325
+10.78303,11.15874
+9.31380,-9.56016
+-9.79354,-10.74205
+-10.20976,10.55071
+9.69320,7.06139
+10.85431,-10.21258
+-11.62342,-8.84518
+-10.25611,7.38299
+10.14933,12.36433
+9.10186,-10.85309
+-10.52361,-10.63383
+-10.27399,10.02691
+8.64972,10.01966
+7.13548,-11.04442
+-9.28979,-9.01762
+-10.86134,9.65272
+9.94366,10.29090
+10.13123,-9.34303
+-9.53558,-11.16861
+-10.75521,10.02608
+9.09291,8.78215
+10.33433,-10.31838
+-9.31069,-10.92428
+-11.31159,10.34519
+12.17999,9.66812
+9.53391,-10.64596
+-9.40312,-9.06542
+-7.98380,11.44235
+10.59108,11.66770
+10.52200,-9.69516
+-10.79671,-9.86056
+-10.29328,11.79981
+8.13349,9.71739
+10.23289,-8.46116
+-11.11053,-9.36184
+-8.97241,9.60279
+10.75944,8.68592
+8.91334,-10.12121
+-9.17715,-9.77352
+-11.94707,10.95491
+8.68308,10.81667
+9.11024,-9.39862
+-8.93691,-9.17645
+-9.20689,11.01444
+9.59135,11.27653
+9.72923,-9.11236
+-9.01988,-9.07482
+-9.42046,8.45712
+10.22627,10.06318
+10.90934,-10.92308
+-9.12129,-11.12259
+-10.07268,11.02470
+10.06780,10.40607
+10.58211,-9.44782
+-12.31571,-11.63850
+-11.88045,8.34354
+9.91000,8.52215
+9.46662,-11.21716
+-8.22553,-10.73455
+-11.71477,11.56907
+11.02730,10.26451
+9.10419,-10.55651
+-11.17340,-10.86840
+-11.49377,10.44853
+10.20998,9.72075
+8.66162,-9.27312
+-9.04373,-8.68571
+-10.63072,10.84248
+9.20017,11.35085
+9.46735,-10.23793
+-10.97695,-9.08959
+-11.21667,8.20108
+9.84028,10.71676
+8.68418,-8.63458
+-10.14569,-9.48951
+-10.64585,8.87767
+10.83029,9.07718
+11.48718,-10.04883
+-9.44055,-10.91378
+-9.32600,10.31297
+9.21373,11.94009
+9.18402,-9.12363
+-8.44338,-8.96281
+-9.81657,9.16753
+10.78603,8.91728
+9.02824,-10.65734
+-10.55005,-8.24239
+-10.21779,10.56365
+9.19810,10.38685
+7.61372,-10.11039
+-10.26634,-10.93470
+-10.89266,8.52745
+9.71957,11.53867
+9.96786,-10.65191
+-10.52605,-9.78439
+-11.39275,9.59631
+11.57090,10.54540
+10.04247,-9.70921
+-9.82970,-10.20287
+-9.19347,10.67985
+9.35496,11.07946
+9.56822,-10.02363
+-11.82745,-9.96217
+-9.03433,8.64249
+12.03483,9.75404
+9.17975,-9.52373
+-11.38407,-9.89603
+-12.00043,7.76869
+9.64574,8.42006
+9.90632,-10.97076
+-10.24289,-10.99370
+-9.02978,9.00852
+9.14838,8.49758
+9.09464,-10.34229
+-8.47329,-10.54345
+-8.25904,9.78890
+9.77277,9.15306
+10.64213,-9.83804
+-12.21470,-11.05765
+-11.36959,8.68802
+9.42619,9.71206
+9.65592,-10.62472
+-9.07962,-9.55955
+-8.46749,9.80527
+8.07183,12.29033
+8.84076,-10.67166
+-9.94555,-9.25282
+-9.01231,10.60756
+9.39164,10.38731
+9.00185,-12.25021
+-8.87054,-9.26871
+-11.09259,9.14308
+9.12394,8.17206
+9.14974,-10.52855
+-8.96124,-9.81307
+-8.72395,9.01681
+9.76189,10.29099
+10.69675,-10.20384
+-12.07599,-10.18019
+-10.26466,10.71940
+10.08463,8.82281
+9.99190,-9.30681
+-9.14612,-10.00634
+-11.70568,11.19155
+9.55217,11.20509
+10.12257,-10.08786
+-8.81994,-10.30928
+-10.48227,10.41662
+10.79589,8.18523
+9.93309,-10.46142
+-8.52549,-10.56901
+-10.35082,9.73686
+9.39180,10.40914
+8.56561,-10.02011
+-10.66207,-10.94041
+-9.15114,9.32917
+10.35774,9.83830
+10.76170,-9.46556
+-9.39251,-10.56503
+-10.32763,8.90293
+9.76491,10.62985
+11.02513,-10.65088
+-9.84368,-10.92515
+-10.80574,8.74563
+9.91962,12.75842
+9.36783,-9.87145
+-10.54974,-9.38346
+-10.44037,8.12341
+9.76277,9.79711
+9.93353,-10.17968
+-9.03070,-7.79841
+-10.91163,9.24894
+11.12624,11.62753
+10.74169,-10.25744
+-10.63551,-10.44735
+-10.66288,9.86624
+8.97522,10.56728
+10.39540,-9.05180
+-10.09884,-9.18351
+-10.22327,12.03472
+10.45669,9.52151
+9.29568,-9.73897
+-8.71884,-12.15683
+-10.95612,9.94773
+11.10916,8.98165
+11.26034,-9.44006
+-8.56762,-11.53060
+-9.58490,11.43778
+9.93042,13.09783
+11.30762,-11.92921
+-11.66351,-9.00012
+-9.18069,10.58088
+11.38248,9.92744
+9.87697,-9.53135
+-11.33113,-9.31020
+-10.27772,9.00013
+9.33225,8.57886
+9.05691,-11.14319
+-9.32815,-9.45201
+-8.86360,10.55006
+9.00250,9.45986
+8.71384,-9.97169
+-9.60343,-11.39680
+-8.49407,10.57210
+10.89807,8.00815
+10.22015,-9.11164
+-11.22461,-11.04475
+-9.92957,9.57765
+10.00174,9.09712
+9.61134,-8.88209
+-8.76675,-9.19564
+-10.64011,10.52982
+10.00584,10.06474
+9.21186,-9.90137
+-10.22128,-9.99007
+-10.19926,9.42111
+9.71659,10.69629
+12.55523,-11.00231
+-8.76283,-10.48465
+-8.98374,10.00070
+8.91680,10.15569
+9.17901,-11.31083
+-8.95743,-9.36647
+-11.04135,9.89131
+11.04738,10.90147
+10.70934,-10.26888
+-10.18407,-12.22210
+-9.37183,10.45993
+11.73535,10.35313
+8.94457,-10.13869
+-9.65611,-10.41722
+-9.61205,9.28191
+9.00946,9.57628
+12.68033,-9.44542
+-10.41742,-10.84850
+-10.95729,10.57044
+10.07075,9.44416
+11.11219,-10.38101
+-10.43231,-10.38806
+-9.98210,11.44953
+7.65175,10.57413
+9.96134,-10.74546
+-12.08976,-7.73518
+-9.01159,8.37335
+8.46170,6.91257
+9.14594,-9.39074
+-11.09904,-10.14029
+-10.83930,11.67671
+11.23497,8.79972
+9.12057,-10.02686
+-10.38201,-9.18635
+-9.17191,10.01726
+11.12909,10.61887
+11.42295,-9.70047
+-10.20304,-10.10926
+-9.80433,6.86765
+9.31883,10.49349
+9.75755,-8.01855
+-9.42809,-9.05368
+-8.16285,10.91904
+7.47024,10.76998
+9.22002,-10.68710
+-11.74784,-10.62674
+-11.10775,10.30568
+9.95982,9.24919
+10.34863,-10.81285
+-9.38154,-8.92884
+-9.53100,9.51866
+8.81763,9.56066
+11.35097,-9.66965
+-10.31205,-9.55240
+-11.12370,9.58837
+11.08581,9.83670
+11.33839,-10.37767
+-11.35763,-7.62840
+-9.81521,10.56016
+8.57841,12.91430
+8.59866,-9.86888
+-10.62085,-9.86607
+-10.31987,10.85392
+11.28116,8.98298
+9.62795,-9.79008
+-9.91374,-9.28070
+-10.36627,9.72235
+9.66523,10.23579
+10.90999,-11.52749
+-9.64320,-9.28576
+-10.69367,8.71489
+9.99143,10.67020
+10.28055,-8.79470
+-10.88129,-11.17941
+-11.55972,11.80814
+10.21675,9.98334
+9.03624,-9.86100
+-9.37683,-9.63181
+-11.95939,10.02259
+8.28440,9.69232
+10.73603,-9.68715
+-10.19597,-8.99516
+-8.55878,8.19240
+9.87676,7.61994
+10.28918,-9.44181
+-10.29598,-9.01445
+-9.85907,9.62468
+10.62756,10.27969
+9.50859,-7.94121
+-11.33833,-10.06566
+-11.04606,10.68234
+11.64139,11.34194
+10.36004,-10.60432
+-8.93365,-9.75987
+-9.05206,9.06057
+10.34142,9.95585
+9.81867,-9.14117
+-10.20270,-9.58814
+-10.30806,10.14340
+11.52989,8.11707
+8.88789,-8.55926
+-10.37642,-10.50793
+-9.14355,10.85518
+9.08647,10.64592
+9.93965,-10.11981
+-8.68563,-10.18512
+-8.63902,9.97843
+9.64590,9.43137
+10.14961,-8.67646
+-9.17641,-8.81984
+-9.34729,9.77118
+8.95421,11.59008
+8.95888,-9.94391
+-9.12283,-11.93544
+-10.06420,9.95825
+11.09313,8.17985
+9.68128,-11.21016
+-9.10518,-8.82347
+-10.21815,11.07376
+11.21551,11.26219
+10.15155,-12.08562
+-10.64292,-10.91090
+-9.37556,11.08204
+10.95132,10.13794
+8.67895,-8.31105
+-9.52649,-10.63761
+-9.63236,11.37789
+10.23234,11.62791
+8.59844,-7.84726
+-9.47084,-10.23203
+-10.12836,10.29746
+10.13075,9.39909
+10.22950,-11.37860
+-9.69108,-11.21652
+-10.66225,11.18882
+10.17039,9.70560
+10.01420,-9.65531
+-8.82346,-10.99231
+-8.29703,10.78639
+10.58583,9.71705
+12.39152,-8.68903
+-10.83382,-8.90271
+-9.54889,8.83468
+10.54805,9.75070
+10.05538,-10.32031
+-10.60349,-11.50906
+-7.41607,10.42236
+10.19934,10.16996
+10.86629,-11.45158
+-10.21020,-8.27410
+-10.61642,10.31113
+12.50840,9.33090
+8.98524,-9.46748
+-8.57931,-9.75056
+-11.18008,10.11613
+10.44066,9.89595
+10.50743,-11.19150
+-10.56792,-10.07886
+-10.31499,9.26391
+10.18725,11.21930
+8.53538,-9.95245
+-10.39934,-9.71203
+-8.10867,11.21541
+9.30601,10.22532
+9.57329,-9.79952
+-10.97127,-8.07082
+-9.18454,10.14768
+10.77432,9.73359
+9.52374,-10.78876
+-10.44938,-10.64027
+-11.27428,10.68999
+10.09469,10.62991
+11.39931,-8.75052
+-11.23058,-10.93612
+-8.61358,10.42629
+9.80687,9.20835
+11.36602,-7.82430
+-8.74741,-7.60721
+-9.41937,9.06734
+7.23343,10.19085
+10.01489,-11.12728
+-10.01190,-9.81359
+-9.45411,9.20715
+10.54315,11.62020
+9.42288,-9.41099
+-10.88504,-11.56657
+-8.87498,9.07195
+10.44461,8.53199
+11.48029,-12.08140
+-10.46018,-8.93027
+-11.16578,10.84984
+9.34688,10.13456
+10.00060,-8.84485
+-8.71055,-10.43194
+-10.06400,9.81735
+12.46136,12.10284
+10.09039,-8.90072
+-9.14420,-8.43887
+-10.27311,11.12702
+9.71678,8.68453
+9.79119,-8.49853
+-9.53595,-11.26695
+-9.35021,8.75942
+10.33747,9.08456
+9.59569,-10.86780
+-11.63369,-8.37669
+-10.70938,10.27860
+10.18246,9.59653
+10.61940,-10.08698
+-11.26182,-10.59525
+-11.40706,9.91544
+9.94376,10.75739
+11.42533,-8.92881
+-9.31295,-10.46922
+-11.01601,9.59252
+9.56077,10.36101
+10.13344,-9.16996
+-10.75352,-9.03309
+-11.01014,9.46554
+9.35244,9.16038
+9.44069,-8.82463
+-8.06091,-8.54441
+-9.77569,9.23476
+11.40287,10.74700
+8.77214,-7.36422
+-9.90467,-9.48012
+-12.83970,10.70005
+11.01715,10.44226
+10.24266,-10.59150
+-9.53112,-9.43888
+-9.40117,10.13417
+10.38091,9.84143
+8.66077,-10.50337
+-10.29857,-10.31075
+-9.92703,11.14320
+9.01082,9.71279
+8.75710,-10.68376
+-10.00304,-10.03790
+-11.33585,8.79657
+9.45884,9.78705
+9.85303,-10.51929
+-9.57038,-11.16302
+-8.64728,8.98911
+10.97794,7.53266
+11.23245,-10.29494
+-8.88831,-10.65108
+-9.63793,10.90135
+10.73193,8.93260
+10.18378,-9.86079
+-9.39115,-7.98435
+-11.90862,8.87707
+12.20115,10.34295
+9.24390,-11.00948
+-10.62057,-12.03685
+-9.26828,9.55474
+10.29897,10.21953
+8.95536,-9.19739
+-8.12682,-9.94623
+-11.43493,10.83122
+9.81764,9.28805
+10.92873,-9.37619
+-10.87528,-9.64611
+-10.67046,11.05794
+8.93153,10.25580
+11.36975,-7.83705
+-11.63945,-9.70425
+-11.74237,10.41348
+11.12861,11.34377
+9.54949,-9.41152
+-9.39230,-9.88484
+-9.86159,10.58596
+10.21530,9.23195
+8.75884,-10.76544
+-11.78115,-11.30475
+-11.13952,10.31046
+11.73884,9.78785
+10.71853,-10.20984
+-8.53851,-8.45043
+-10.83170,9.86427
+9.60410,10.98711
+9.45746,-10.71558
+-10.74602,-10.26949
+-9.26073,9.73675
+8.83779,11.26201
+10.45080,-11.13297
+-10.25299,-11.22818
+-10.94457,12.14323
+10.28860,8.60598
+9.88022,-10.35037
+-10.74239,-9.94573
+-9.48911,10.55854
+9.09077,9.15323
+8.73250,-11.19189
+-12.06350,-11.13483
+-10.06076,9.81450
+9.22871,8.51944
+8.92191,-10.91863
+-8.50716,-9.03126
+-12.33541,11.32349
+10.67741,11.51271
+9.49507,-7.79442
+-9.51080,-10.37546
+-10.88198,8.31165
+10.79254,9.44799
+9.82311,-10.20879
+-9.81362,-9.53984
+-11.81647,10.69446
+10.37056,9.14510
+10.90866,-10.60759
+-10.10188,-9.22499
+-9.27241,9.69431
+9.53381,10.36576
+9.16279,-10.58318
+-10.18504,-11.32680
+-8.98610,8.49602
+11.59346,11.19848
+10.89081,-8.45550
+-9.68634,-10.91642
+-10.14274,10.02457
+9.79231,8.85506
+7.62982,-8.47550
+-9.36514,-9.07097
+-9.93641,10.09829
+8.85926,9.46097
+8.13496,-11.90115
+-11.06746,-9.54357
+-8.38905,9.86828
+10.27690,9.42094
+10.16973,-9.84956
+-9.44309,-12.61525
+-10.11654,9.34827
+9.72314,10.61842
+10.84465,-10.08898
+-10.89137,-9.59605
+-9.66045,11.29727
+7.73635,10.31017
+9.67015,-10.57757
+-11.55300,-11.55482
+-9.78989,9.96913
+9.89142,13.37064
+8.82103,-8.05648
+-10.37338,-10.39081
+-8.01898,10.90275
+9.27761,9.17426
+8.42192,-10.02913
+-9.06783,-10.29442
+-7.94414,10.21037
+9.66454,10.11785
+10.28427,-10.86977
+-9.32402,-8.24966
+-9.08511,10.30607
+9.81564,9.50388
+8.83976,-11.77567
+-10.90129,-10.11166
+-9.73115,9.36984
+10.02227,8.27614
+10.25611,-9.28844
+-8.13725,-8.90356
+-10.03930,9.71915
+9.42829,9.72893
+8.10481,-9.49083
+-10.13549,-9.37917
+-9.88569,11.32244
+10.29147,11.60076
+10.25656,-9.26858
+-12.66073,-10.96463
+-10.16369,10.70210
+10.23409,10.81135
+8.10045,-8.62960
+-9.78448,-9.48711
+-10.71381,10.07908
+10.03868,8.84286
+10.26740,-8.95698
+-7.46358,-9.45091
+-9.82228,10.46756
+8.39943,10.01157
+8.35755,-7.90150
+-9.49999,-10.17537
+-9.61328,9.90134
+7.64823,9.80835
+10.06141,-9.51567
+-7.81212,-10.23787
+-9.48736,9.31905
+10.29238,12.15529
+10.40824,-10.29085
+-8.83231,-7.60209
+-10.56524,8.57291
+10.05800,9.62145
+10.43961,-8.01405
+-10.43992,-9.82103
+-9.12623,10.46643
+8.22791,10.90174
+9.69249,-10.32059
+-10.97982,-10.13814
+-9.71965,10.22097
+10.79896,9.38567
+10.92866,-12.05467
+-11.08359,-10.39127
+-12.05803,8.96900
+11.90954,10.22247
+11.53059,-9.47952
+-8.50309,-8.40173
+-9.19180,10.60885
+10.73087,11.13469
+9.98407,-9.27016
+-11.00056,-9.65751
+-9.63863,9.43082
+10.11783,8.63446
+9.74274,-9.16266
+-9.97626,-10.05867
+-10.26174,10.40290
+9.96298,8.25211
+9.57557,-9.21700
+-9.56455,-9.19313
+-9.08399,8.80140
+10.71267,9.47465
+8.57486,-9.27685
+-9.30740,-10.46305
+-10.10151,10.63236
+9.43864,7.80785
+10.25963,-9.22998
+-9.91878,-9.45434
+-7.49240,10.63181
+10.19008,12.46773
+11.27027,-11.79871
+-10.02975,-10.08071
+-10.30120,10.83436
+10.09493,10.77429
+11.34207,-9.55733
+-10.36656,-9.52412
+-10.52503,8.90054
+9.52138,10.29582
+9.83866,-9.94166
+-10.98495,-10.67011
+-9.31568,9.72104
+9.04835,9.90322
+8.72617,-10.60773
+-10.99748,-10.36828
+-10.59320,10.26655
+10.51903,9.41487
+10.61160,-10.15676
+-11.48915,-10.42516
+-9.05362,9.48522
+10.21408,9.18807
+10.79083,-10.73356
+-9.98715,-7.99068
+-10.15960,10.71799
+9.49027,10.96694
+8.72413,-9.79222
+-10.51882,-9.64447
+-8.90421,10.11481
+10.53758,9.30715
+8.79031,-10.62354
+-10.44860,-10.22021
+-9.41630,9.69565
+10.49185,9.78083
+10.41236,-10.74951
+-8.99995,-11.24055
+-10.07329,9.42097
+11.13704,10.95294
+10.87527,-10.09367
+-9.77332,-11.14865
+-10.03798,8.28538
+8.88417,9.74308
+9.69993,-8.97554
+-10.26537,-9.51517
+-11.61897,10.91293
+11.42745,7.97281
+10.16114,-9.67549
+-9.13913,-9.05375
+-9.09124,10.01553
+8.30173,10.32023
+11.09691,-9.30661
+-10.26221,-9.37420
+-10.26263,9.25844
+9.92360,10.59916
+11.10387,-9.58636
+-8.94786,-9.63348
+-9.26916,10.36023
+10.09878,10.81285
+12.43935,-10.65763
+-8.88687,-10.84301
+-9.70088,9.48735
+9.40954,9.77851
+9.93787,-9.97246
+-8.01311,-7.97115
+-10.39813,7.96942
+10.18585,9.19204
+8.58927,-10.73614
+-9.94687,-9.69389
+-9.79661,10.49542
+8.96470,8.05799
+8.86697,-10.46352
+-8.14140,-9.01867
+-8.89906,9.65471
+10.05859,10.33832
+9.90064,-10.47071
+-9.97382,-9.52582
+-10.84677,9.02017
+11.29299,8.71382
+9.75358,-10.88602
+-10.74843,-10.26944
+-8.22312,11.32496
+7.78928,6.85491
+9.30930,-11.11741
+-9.08438,-10.34801
+-10.36020,9.63371
+10.74593,8.98624
+11.39959,-9.68283
+-8.61528,-8.72842
+-9.34669,9.62477
+9.23149,9.46548
+10.05078,-11.48159
+-11.87033,-11.50537
+-10.12470,10.67950
+10.57870,9.69781
+9.57776,-10.30966
+-8.95483,-9.80154
+-10.03615,10.62501
+9.27602,8.69073
+10.41064,-10.51582
+-8.95795,-9.75138
+-10.68372,9.94034
+10.75770,11.10525
+10.42548,-9.51057
+-10.61499,-8.76924
+-9.56969,11.65904
+9.26153,9.72936
+8.70317,-8.55022
+-9.36035,-11.28973
+-8.30652,8.86814
+7.38796,10.81219
+9.74214,-9.24163
+-10.07487,-9.08925
+-8.80493,10.89238
+9.87187,10.17476
+9.52211,-9.92273
+-8.37973,-8.95507
+-8.90343,7.50195
+9.31847,9.58836
+9.11175,-8.70747
+-10.21129,-10.63055
+-10.01715,8.07859
+9.57974,8.46462
+10.45297,-10.56136
+-9.34187,-9.97587
+-10.34324,10.20447
+11.21473,10.16910
+8.48989,-9.46523
+-10.82021,-11.05838
+-9.04375,9.28966
+8.45850,9.66386
+9.66666,-11.74032
+-12.00913,-11.56197
+-10.05861,9.38406
+10.16645,9.72489
+10.14765,-11.93362
+-11.03425,-7.58558
+-8.69724,8.82636
+10.43977,11.14242
+10.16478,-11.53699
+-10.56205,-9.56604
+-10.29416,10.00204
+10.04593,11.54082
+9.17699,-8.24008
+-11.24587,-10.82868
+-11.77919,9.18867
+11.00123,9.60507
+10.62138,-9.52046
+-11.18859,-10.80386
+-9.98550,8.36760
+9.27854,10.64771
+9.57637,-11.10327
+-10.27560,-11.70181
+-10.78063,9.66830
+9.01321,11.31550
+10.04706,-9.14451
+-10.61538,-9.25003
+-10.72776,10.39826
+10.00115,9.05411
+10.05201,-10.01925
+-10.62021,-10.29563
+-8.52157,9.04720
+10.33554,10.89161
+10.73051,-9.77846
+-9.54022,-10.41126
+-11.93129,9.08781
+8.86654,10.52514
+9.95291,-9.53569
+-10.26526,-10.36316
+-9.98191,9.53464
+10.77302,10.61661
+10.34878,-9.71436
+-8.57663,-9.25774
+-9.16810,9.95745
+10.61501,9.20390
+9.91982,-9.66013
+-8.87431,-11.50352
+-10.33544,9.35380
+8.96876,11.69767
+11.31061,-8.99892
+-11.64792,-9.71480
+-8.97016,10.66570
+8.94675,8.72602
+10.10665,-10.59308
+-7.77813,-10.48148
+-9.55935,10.23494
+9.48400,9.41565
+9.26340,-9.65629
+-9.26824,-10.09440
+-9.33359,11.64619
+9.32130,10.53831
+9.66186,-8.09057
+-10.55106,-9.72915
+-9.13773,8.56748
+9.91420,8.88461
+11.87121,-9.19795
+-8.52444,-9.69319
+-9.75398,12.44008
+9.36625,8.49667
+9.93130,-8.76524
+-10.31337,-10.82602
+-10.98234,10.28975
+10.02965,10.26040
+10.20142,-9.86858
+-10.07231,-11.44839
+-9.82987,9.06802
+13.77523,11.11911
+8.49006,-9.43273
+-8.65297,-10.02517
+-8.70526,9.62156
+10.45135,10.76191
+8.57007,-9.22906
+-9.72671,-9.24253
+-9.06986,10.59022
+12.19531,9.96235
+9.35803,-13.17950
+-10.23794,-9.77125
+-10.49353,9.74633
+9.14935,9.60927
+9.29166,-10.42316
+-12.06601,-10.51670
+-9.76885,10.30785
+10.80934,9.70082
+8.90386,-9.88466
+-11.43371,-9.87578
+-9.68677,9.22946
+9.97022,10.61899
+9.58628,-9.37987
+-9.34280,-11.43018
+-10.04837,12.78847
+7.16937,10.68196
+9.68787,-8.29797
+-8.93274,-8.60619
+-9.59582,12.74637
+11.25458,7.33749
+11.05110,-9.63549
+-10.41344,-10.97416
+-12.40197,9.70911
+11.71486,8.96421
+9.26407,-10.02814
+-11.89671,-10.73243
+-10.78305,10.00215
+9.76740,10.05167
+10.52326,-9.52343
+-10.98074,-10.50207
+-9.62023,8.80654
+10.67449,11.86048
+9.75343,-10.96863
+-7.71012,-7.53490
+-8.43112,10.70940
+10.47901,8.36249
+8.75658,-10.91367
+-10.49724,-9.26809
+-10.69176,11.37795
+9.02868,11.09194
+10.79565,-8.58827
+-11.32470,-9.71137
+-10.20600,11.19200
+11.72638,10.07389
+11.36365,-8.37752
+-11.36400,-10.96148
+-10.42149,9.98923
+10.26037,9.51906
+10.09895,-7.77775
+-11.13409,-8.68685
+-11.22453,9.26042
+11.34873,10.51008
+8.77007,-10.23193
+-9.14131,-8.46249
+-9.96738,12.13054
+11.24427,9.54387
+10.24806,-10.44481
+-9.82907,-9.97200
+-8.29856,8.17933
+9.35122,10.66247
+10.12723,-11.85894
+-7.99299,-9.90126
+-8.90097,8.81442
+9.87689,10.38579
+10.54469,-10.18386
+-10.54740,-11.11530
+-10.68769,9.35346
+10.26228,11.08989
+12.15735,-8.46826
+-12.23069,-10.15796
+-10.52164,10.08772
+10.22517,11.06581
+11.37365,-10.12747
+-9.62778,-8.57673
+-9.26155,9.69960
+11.29957,8.88151
+10.46351,-10.36941
+-9.86949,-10.77862
+-9.60911,11.06406
+11.52239,10.25114
+10.68102,-10.97819
+-10.69221,-9.74165
+-11.02931,9.25182
+9.77004,10.43439
+8.41721,-10.03259
+-8.72851,-10.62940
+-9.36870,11.88402
+10.27412,8.67023
+9.47240,-11.84806
+-9.93342,-10.19328
+-10.33703,10.87346
+11.74315,9.44902
+9.59019,-9.15581
+-9.41675,-11.70544
+-10.91377,9.17930
+10.54523,10.43051
+8.38783,-8.90797
+-11.53979,-9.66409
+-9.08613,9.69398
+9.36211,11.86250
+10.01527,-9.24198
+-9.72848,-11.80463
+-9.95779,8.64191
+9.26235,11.20986
+10.53754,-9.86706
+-9.50807,-9.57888
+-9.41558,10.91265
+11.06314,9.49055
+10.40064,-8.46464
+-10.30367,-9.17482
+-11.37709,9.05956
+9.20260,9.33063
+10.63382,-10.70928
+-10.05013,-10.95712
+-11.10677,10.49603
+9.23416,10.69659
+10.64805,-10.79946
+-9.91327,-10.26996
+-11.54094,8.97869
+9.88196,10.20565
+10.03565,-9.34342
+-9.17323,-8.51176
+-9.73101,8.05347
+10.94520,9.86202
+10.90870,-10.73552
+-8.62379,-8.64847
+-9.24582,10.02914
+9.23669,9.12268
+9.49233,-10.35312
+-9.63243,-10.41489
+-9.88397,8.43372
+10.74400,9.53712
+10.91033,-8.84003
+-8.76455,-11.36476
+-8.51776,11.65391
+10.77358,9.87754
+8.93782,-10.64319
+-10.06938,-9.53143
+-9.13760,9.97466
+10.49995,12.35054
+9.59943,-8.15254
+-10.84809,-10.29013
+-12.47266,10.27557
+7.96058,11.56023
+11.02070,-8.72856
+-8.85972,-10.62530
+-9.51982,10.40116
+10.55778,11.88184
+9.30673,-9.82976
+-11.27503,-9.85993
+-10.14500,10.34902
+10.22621,12.31168
+10.76042,-12.28561
+-9.06728,-9.00747
+-9.29917,10.12530
+9.51122,11.16282
+10.27653,-10.34604
+-8.59629,-10.04606
+-10.99511,8.37113
+9.81553,11.94851
+11.22228,-10.18903
+-8.41582,-11.65123
+-10.43897,11.43145
+10.96359,9.84408
+11.76582,-9.52572
+-9.80338,-9.04725
+-9.65512,10.95587
+10.63507,10.91909
+9.92995,-10.56163
+-11.44536,-10.71893
+-9.32953,10.55971
+9.52930,9.88282
+9.60176,-11.46181
+-8.93272,-11.56964
+-9.28164,11.14048
+10.80766,9.30991
+11.34223,-9.08049
+-12.52020,-10.08738
+-11.05708,11.41040
+9.91112,8.50462
+10.15970,-9.96931
+-11.65162,-10.25400
+-10.17328,8.71856
+8.97450,10.87699
+9.66180,-10.88332
+-10.73438,-10.05119
+-9.01206,9.47178
+9.20011,10.58826
+9.20722,-8.34251
+-10.81765,-9.92251
+-9.67992,10.96514
+9.36658,9.68190
+10.06136,-11.47935
+-9.24651,-9.14809
+-10.06712,12.08051
+8.72314,9.23254
+10.15590,-10.08945
+-10.49574,-10.66607
+-10.65401,11.51609
+10.82359,10.52872
+10.26793,-10.75445
+-10.63152,-8.27071
+-9.24742,11.63616
+10.46479,9.45771
+9.83943,-9.41169
+-9.81159,-10.80172
+-8.76300,9.04612
+9.13289,10.77508
+10.33953,-11.00268
+-10.48633,-10.75519
+-9.48298,11.41786
+10.94512,8.98054
+10.61772,-10.94239
+-10.51447,-10.20656
+-9.68553,10.33960
+10.51765,9.17161
+9.27185,-9.27043
+-11.02994,-11.46298
+-10.41723,9.21392
+10.69421,9.69546
+9.76734,-8.16965
+-9.64725,-9.71798
+-8.19747,10.03941
+8.74976,11.92392
+9.93974,-10.99326
+-8.82209,-9.37000
+-10.73201,10.87831
+9.94506,9.55683
+8.78306,-8.11815
+-9.55108,-11.10106
+-9.28452,10.62181
+9.37640,9.62861
+9.58419,-9.52580
+-10.71463,-8.70406
+-9.22980,11.09672
+11.21220,11.20464
+9.25768,-9.09784
+-10.74487,-10.87442
+-11.39669,10.70390
+10.05141,9.81114
+11.72774,-10.32442
+-9.15169,-10.27653
+-10.15300,9.93232
+9.58731,9.75050
+8.50882,-10.78599
+-11.00586,-10.31488
+-9.99149,9.09220
+10.55675,10.90932
+12.41199,-11.38180
+-9.60921,-9.33850
+-8.30484,9.49961
+9.40663,7.72001
+9.29419,-9.04519
+-10.89632,-9.48067
+-11.11663,9.79001
+8.97973,11.32288
+10.75832,-9.15628
+-11.46512,-9.00557
+-10.36416,10.86222
+10.33201,9.90811
+10.05396,-8.33248
+-9.46666,-9.30906
+-10.79713,12.80420
+8.74720,9.63052
+7.22428,-9.90212
+-10.73249,-11.32450
+-10.85265,9.68724
+9.60742,11.00674
+8.76441,-9.99641
+-9.43289,-10.89242
+-10.77803,10.45068
+8.72612,10.33196
+10.63127,-10.35650
+-9.62873,-10.11315
+-10.57074,9.62180
+9.84751,10.33827
+12.54204,-8.62895
+-8.50906,-11.09035
+-11.05006,9.14876
+9.47924,9.59823
+9.13331,-10.15647
+-10.77882,-10.04661
+-9.31677,9.80668
+8.56435,11.01034
+9.48254,-9.45925
+-10.71666,-9.66244
+-8.32123,10.14446
+10.66998,9.82960
+10.95518,-10.50138
+-8.07116,-9.83334
+-10.30411,8.03502
+9.81299,11.09427
+10.46350,-10.34072
+-9.12515,-8.87737
+-8.72974,11.30472
+9.55612,9.94261
+8.65895,-9.20654
+-11.31139,-10.69582
+-10.11088,10.61654
+10.83594,10.14452
+7.52893,-9.52547
+-10.08782,-8.74921
+-10.67685,8.48883
+10.66249,9.82324
+11.23981,-10.94864
+-10.66636,-8.23692
+-8.28320,11.00097
+11.19420,11.01949
+11.57682,-9.12216
+-9.92540,-9.51972
+-8.25722,9.82577
+9.67777,10.97970
+9.46507,-7.72582
+-10.58347,-9.26546
+-9.56745,10.65964
+11.32679,9.84512
+10.53445,-10.40011
+-10.97230,-10.05766
+-10.87538,9.33986
+10.55011,9.86527
+10.95034,-10.23911
+-11.17194,-11.15350
+-9.61023,9.94043
+10.84756,9.54282
+9.91944,-9.38367
+-9.87051,-8.88112
+-9.11236,10.33356
+9.85076,10.47811
+10.32324,-9.49815
+-9.56949,-11.21792
+-10.18872,10.38557
+10.13506,9.54057
+9.97728,-8.58245
+-8.96334,-9.61202
+-9.42823,11.21769
+10.23562,10.07768
+9.22572,-10.25067
+-10.84741,-8.56116
+-11.15524,9.64426
+9.73395,10.98061
+9.66352,-9.39729
+-9.16182,-9.36796
+-11.15496,10.53398
+11.10256,10.36998
+9.23055,-10.32539
+-9.16658,-12.31291
+-9.37146,10.34746
+9.07057,9.95086
+10.97870,-9.53539
+-8.49468,-8.58409
+-9.22899,9.93242
+11.38776,10.00956
+11.09739,-10.32275
+-8.16103,-10.95242
+-9.87349,9.85633
+9.09874,10.49119
+10.54493,-12.34582
+-10.01911,-11.41565
+-8.11921,9.05987
+9.80882,10.38891
+9.50415,-11.67380
+-9.41809,-10.62651
+-9.89105,8.60033
+8.81263,9.44906
+9.12082,-10.29062
+-11.61150,-8.79754
+-10.42527,9.95621
+9.25015,10.68088
+10.59286,-9.88131
+-9.76403,-9.63709
+-9.49757,10.39511
+10.79436,10.82146
+10.31903,-9.68409
+-9.29293,-9.02558
+-10.39399,9.77293
+9.70462,9.66431
+10.41360,-7.92108
+-10.12486,-11.93206
+-9.35932,8.54749
+10.14842,9.41846
+11.76389,-11.90797
+-10.08750,-9.65788
+-9.97838,9.76057
+9.75586,10.98057
+10.91846,-9.55232
+-11.70195,-10.14839
+-11.28267,10.20158
+10.04564,10.93985
+9.65993,-9.06430
+-10.12082,-9.94636
+-9.58580,9.75788
+9.80034,9.77837
+10.88168,-11.03571
+-10.20231,-11.36585
+-8.68889,9.22287
+9.51192,9.32350
+8.08726,-10.23635
+-9.92119,-11.16429
+-10.07708,8.32441
+11.06421,10.86857
+10.00981,-9.43246
+-11.11048,-10.20382
+-10.16736,9.90431
+9.96588,11.33658
+8.47901,-11.51421
+-10.70316,-11.61020
+-9.21544,11.36415
+9.28177,8.75641
+10.26093,-11.07417
+-9.19687,-9.18078
+-9.63598,11.22402
+12.45052,11.66485
+11.71317,-8.85213
+-10.42936,-9.33734
+-9.84386,10.73290
+9.35943,12.00000
+9.02700,-8.93005
+-8.45595,-9.95827
+-10.71916,10.60986
+8.66937,10.48797
+8.58664,-8.99510
+-10.58154,-11.13543
+-8.98823,11.86729
+8.75938,10.20907
+9.38847,-10.24313
+-10.65800,-9.73217
+-9.34904,9.35143
+11.48304,12.30928
+9.54866,-10.83618
+-12.78285,-9.77910
+-8.75789,9.93859
+9.72461,10.32836
+11.46731,-11.89194
+-9.95369,-10.15833
+-6.48753,9.79867
+10.63689,8.88956
+9.65763,-9.55054
+-10.01366,-10.31388
+-12.47651,10.17260
+10.75784,7.50919
+10.67888,-9.94254
+-8.71898,-9.96403
+-10.54756,11.08456
+8.38188,10.60898
+10.24611,-8.52224
+-10.21436,-8.31676
+-10.29323,8.83244
+11.11428,10.94138
+9.57732,-10.86783
+-10.04210,-10.61705
+-9.01558,8.10989
+9.72179,9.38154
+9.21167,-10.51176
+-11.88979,-9.73775
+-8.95058,10.68212
+11.06269,10.48581
+10.50400,-10.10573
+-9.46415,-9.01467
+-9.00983,9.35440
+8.81910,11.27302
+10.88145,-11.01960
+-10.39664,-10.77601
+-9.41464,10.01492
+10.36879,11.55812
+10.19117,-8.85568
+-9.16151,-9.36203
+-9.93160,10.00135
+9.46488,9.62822
+12.08159,-9.67807
+-10.11062,-8.87672
+-9.50830,10.18036
+11.47905,11.07327
+8.93802,-10.21347
+-8.97480,-11.96969
+-10.38040,9.60638
+8.92887,11.08729
+9.09740,-10.61234
+-11.23546,-10.25838
+-10.23165,10.14898
+9.43153,9.61206
+9.69910,-9.15821
+-9.31877,-9.17928
+-9.35976,10.23233
+10.77204,11.11877
+11.00127,-9.62505
+-11.11597,-12.43075
+-11.12302,8.34158
+11.62617,9.32708
+8.27712,-10.61947
+-8.74303,-9.84894
+-9.84345,8.92757
+9.44434,10.88028
+8.90550,-11.36281
+-10.96202,-10.67294
+-9.80208,9.62004
+10.96359,8.49037
+12.57447,-10.99806
+-10.95649,-9.60744
+-8.75930,9.90164
+11.51307,10.81354
+9.73605,-9.09987
+-11.90274,-9.76892
+-12.68902,11.41443
+11.13479,10.69516
+10.56952,-9.81898
+-8.76418,-12.01262
+-10.12590,10.87514
+9.66772,11.35788
+9.29638,-9.83334
+-9.33798,-10.81405
+-9.71455,10.57550
+11.20442,10.78578
+9.65850,-9.38445
+-11.18786,-11.04336
+-11.35029,11.77470
+10.61591,10.01396
+9.65016,-11.40521
+-10.26432,-8.33185
+-11.43987,10.49707
+9.98318,10.95389
+10.90564,-10.75275
+-10.66429,-10.14065
+-11.02420,9.66464
+9.54516,9.67920
+10.62584,-9.29827
+-10.05843,-9.42296
+-11.17494,10.54625
+11.21262,10.44992
+10.41646,-10.04546
+-9.38082,-11.77125
+-9.78977,11.46869
+9.13326,9.65745
+9.08982,-9.41192
+-9.04146,-10.72113
+-10.05527,8.68609
+9.28902,10.62374
+9.96715,-9.52926
+-10.84928,-9.97805
+-9.91134,9.22446
+9.01128,7.35471
+9.67628,-10.03696
+-8.39649,-9.10345
+-8.58357,11.01150
+9.50267,9.81731
+8.34761,-10.29155
+-11.25075,-8.65237
+-9.83897,10.68041
+8.61666,9.73915
+11.29684,-9.62726
+-8.39009,-9.62451
+-10.54310,10.31605
+9.93261,10.91320
+8.08135,-9.80894
+-10.85860,-11.11780
+-10.33204,8.94049
+9.26454,10.70649
+7.69582,-10.96738
+-9.90665,-10.69615
+-11.08703,8.54537
+9.67302,11.43833
+9.66627,-11.63193
+-10.68746,-10.13854
+-11.47941,9.99893
+12.03513,11.46537
+10.23220,-9.35160
+-9.70757,-11.27075
+-10.95212,12.25114
+9.40507,11.07201
+11.06141,-8.87713
+-8.87035,-10.37353
+-8.82804,10.48145
+10.18807,8.74444
+10.92173,-9.60636
+-9.98356,-9.76093
+-9.13420,9.20706
+7.72063,11.64260
+10.33652,-10.20726
+-11.34926,-11.80395
+-10.21474,10.86714
+9.99703,9.09794
+9.31355,-9.08650
+-10.22913,-9.83359
+-10.13945,9.41189
+11.83300,9.38975
+8.76697,-10.17062
+-9.11991,-9.26770
+-10.87636,9.21229
+9.64173,10.72761
+11.51261,-10.62531
+-9.27832,-9.25044
+-9.88727,10.41861
+11.18124,10.81273
+8.76149,-9.75480
+-9.15448,-7.47340
+-9.39772,10.15595
+11.38670,11.39246
+8.85302,-10.02174
+-11.12029,-8.30313
+-7.96083,10.21583
+9.86567,9.89171
+10.02400,-9.73397
+-10.00207,-8.64423
+-10.13799,10.12231
+10.25084,9.13070
+7.95102,-11.67874
+-10.50248,-8.19376
+-9.75231,11.01024
+9.47098,9.58491
+9.09210,-9.83283
+-9.71090,-10.64504
+-10.29958,8.46582
+9.52497,9.78211
+9.10695,-10.65761
+-11.26273,-9.06319
+-8.67246,10.42683
+10.36109,9.92035
+9.84594,-10.28863
+-9.98311,-9.51440
+-10.68153,9.87446
+9.25980,11.71431
+8.45016,-9.26869
+-9.34349,-10.51681
+-11.57937,10.05128
+12.23597,10.03824
+11.25827,-10.31637
+-9.60962,-10.91089
+-8.76093,9.46547
+9.75119,10.12929
+10.94764,-10.99917
+-11.45251,-11.01005
+-9.26641,11.61648
+9.55809,10.22411
+10.28703,-11.80163
+-9.04366,-11.12136
+-9.97184,8.75429
+11.45993,10.77892
+9.68127,-9.12738
+-10.30896,-10.99783
+-8.54114,9.34677
+11.13350,11.03750
+9.74747,-11.27004
+-10.63543,-10.54192
+-10.49505,9.96040
+11.23819,11.51421
+8.88411,-8.92800
+-10.83031,-8.94617
+-9.63712,10.80363
+10.79602,9.69452
+11.15199,-9.09152
+-9.72578,-9.55873
+-8.50657,10.15676
+10.78069,11.30893
+10.40664,-10.07187
+-8.88772,-10.76599
+-7.80501,10.40895
+9.14074,8.50371
+11.46769,-9.78411
+-8.78738,-9.83173
+-8.87956,10.67837
+11.29991,8.93618
+10.68765,-9.65708
+-11.46473,-9.71215
+-9.56675,10.64054
+9.81440,10.65731
+9.94032,-10.17493
+-10.47646,-11.33919
+-8.97375,10.13624
+10.60463,10.71056
+9.23923,-9.97681
+-10.35348,-10.52157
+-8.01824,9.90597
+11.55259,10.57854
+7.85135,-10.38584
+-10.70804,-10.08515
+-9.45703,10.44238
+8.81660,10.62827
+10.29196,-10.86277
+-10.03287,-10.19048
+-9.88497,8.46777
+10.13704,11.62318
+10.21857,-10.04071
+-9.80741,-8.71096
+-9.35334,9.31299
+10.22190,10.11038
+9.91823,-10.80837
+-9.29929,-10.73448
+-8.10509,11.54563
+10.40265,9.62499
+10.04536,-10.67386
+-8.70608,-8.11723
+-8.78058,11.14555
+10.89180,7.63728
+8.75012,-11.73007
+-10.81538,-8.42308
+-9.55336,9.49049
+10.78883,11.73240
+9.41439,-9.39762
+-9.82854,-10.52320
+-10.89400,11.34739
+8.36738,10.13198
+10.61618,-9.84122
+-9.68315,-8.16281
+-9.30442,10.31286
+11.08335,11.02127
+8.06876,-8.08919
+-11.67878,-9.95637
+-10.29299,9.38178
+11.43735,9.64398
+7.07927,-9.59813
+-9.95969,-10.13603
+-9.97238,8.96582
+10.46421,9.34764
+9.58782,-10.89159
+-11.18827,-10.83263
+-9.45759,10.47234
+9.06704,10.67189
+10.69704,-11.30939
+-9.80517,-9.89456
+-10.35597,8.98846
+9.46721,7.81586
+10.48570,-11.17938
+-9.85058,-9.90037
+-9.17536,9.04315
+7.26351,9.72441
+11.19029,-11.01665
+-7.56522,-9.23742
+-10.17064,9.91536
+9.35610,10.88007
+10.06195,-9.52441
+-9.84604,-8.51452
+-8.50234,11.42278
+9.13296,8.86444
+10.00392,-9.47755
+-9.09458,-10.93871
+-9.24770,10.93379
+9.76576,10.04836
+9.39808,-10.50132
+-9.28507,-10.82278
+-11.13884,9.18443
+9.91173,11.09614
+12.68852,-9.35091
+-9.35202,-7.04982
+-9.26936,10.16126
+8.51445,9.34433
+10.43349,-11.01016
+-12.43068,-9.78986
+-10.96970,10.21968
+8.63822,10.90055
+9.68968,-9.19006
+-7.63540,-10.22937
+-9.21565,8.84344
+10.41037,10.00286
+9.24627,-8.55200
+-10.52174,-10.45837
+-9.37823,8.71865
+9.07992,9.67774
+9.19916,-9.70985
+-9.65709,-9.82867
+-9.72942,9.62945
+11.26059,11.60183
+9.76214,-10.12139
+-12.04617,-7.54695
+-11.04367,10.71221
+10.44599,11.32350
+9.70986,-8.41883
+-8.78874,-8.87635
+-9.41039,10.40443
+8.65838,9.35821
+8.57859,-8.39171
+-9.52615,-11.71585
+-9.07534,9.39323
+9.65927,9.37164
+8.90647,-8.99209
+-8.49195,-10.77848
+-9.80143,9.03227
+8.99377,12.46187
+8.71614,-9.57077
+-9.88845,-9.49825
+-9.48021,9.94272
+9.57073,9.14607
+10.24550,-10.52367
+-9.97583,-9.98756
+-9.65166,10.81465
+9.00268,10.08659
+9.99469,-9.01855
+-10.34221,-10.10749
+-11.29958,11.27728
+9.77149,10.89629
+11.39460,-8.31487
+-11.04463,-9.56643
+-10.24057,9.55464
+9.42384,11.03647
+10.82889,-10.89699
+-9.52755,-8.09438
+-10.48211,10.74054
+9.62954,11.06087
+9.89364,-9.41459
+-8.74677,-10.76269
+-11.31658,9.14634
+11.09259,9.99377
+10.97589,-9.75421
+-9.90299,-10.37710
+-9.71072,9.91462
+10.11981,10.13761
+9.18503,-9.77411
+-8.75367,-11.59164
+-10.16332,8.64383
+8.66317,9.49702
+9.20767,-10.35618
+-11.46478,-8.06407
+-7.37919,11.45722
+8.60808,8.37396
+10.28650,-10.58545
+-8.74038,-9.68618
+-9.72142,9.80148
+12.86783,10.35090
+10.95583,-8.76520
+-10.09401,-9.97009
+-11.76504,8.62316
+10.33083,9.91441
+11.12871,-9.79498
+-10.10191,-9.43903
+-9.83611,10.13782
+10.15642,10.99102
+10.64703,-9.88225
+-9.62518,-9.89158
+-9.33738,10.25266
+9.55877,8.40643
+10.41408,-11.61094
+-9.72761,-9.59599
+-11.20050,10.46229
+10.80864,10.32111
+9.88052,-10.47767
+-10.19457,-9.13162
+-12.28361,8.66469
+10.18105,11.03602
+11.44866,-7.85868
+-10.78753,-9.95829
+-9.45806,10.85444
+10.01284,8.88840
+9.89566,-10.25549
+-9.68296,-8.16568
+-10.53076,11.09807
+10.32118,10.28917
+9.34904,-10.77911
+-8.84832,-9.06044
+-12.07716,10.22638
+10.51539,11.21875
+9.93432,-10.00735
+-9.33211,-9.98367
+-9.57940,9.69891
+9.16626,8.89083
+10.83236,-8.25410
+-10.58714,-10.38795
+-10.15722,8.01821
+9.30085,10.31211
+7.87702,-10.40020
+-9.68486,-9.42668
+-10.49802,10.80221
+9.52830,10.52388
+10.10093,-10.28727
+-9.12515,-9.14071
+-12.39513,9.87077
+9.79769,11.51927
+10.19861,-9.91666
+-11.44605,-10.03786
+-11.25194,11.13112
+10.32884,10.44183
+10.44204,-8.36597
+-8.86951,-8.80167
+-10.27929,9.25809
+9.59590,10.03350
+10.55713,-8.66720
+-10.22682,-11.04382
+-8.44709,10.94224
+10.91625,9.42902
+10.18478,-10.80403
+-10.57221,-9.78627
+-11.89830,9.91815
+9.94192,10.45844
+10.47234,-9.27371
+-9.53615,-9.28024
+-9.45243,9.81051
+10.74421,10.64039
+9.48090,-10.84110
+-11.09633,-10.41413
+-10.35536,11.06996
+8.00545,9.56759
+10.74502,-12.36511
+-9.98506,-7.61184
+-9.07358,10.55553
+10.00843,10.17667
+9.40756,-10.01304
+-10.03595,-10.38492
+-12.25822,10.29074
+10.79379,8.49146
+9.39130,-9.22067
+-10.44264,-9.94976
+-11.46935,10.56171
+9.83335,8.47096
+11.87404,-10.26552
+-9.44567,-10.10980
+-10.61882,10.05741
+10.19215,10.89152
+9.23170,-9.81418
+-10.82481,-7.60320
+-10.84980,10.09805
+8.81936,10.67872
+10.41991,-9.65624
+-11.76887,-11.16617
+-8.76852,8.76763
+8.09737,10.75104
+10.34076,-9.72384
+-9.88436,-9.34529
+-10.67153,10.56245
+10.52263,8.61026
+10.05697,-10.07694
+-9.42612,-11.25418
+-9.85561,7.75542
+10.52922,9.84579
+11.34687,-9.34851
+-11.16798,-10.04400
+-11.40734,10.48720
+10.01353,8.75841
+11.03928,-9.18075
+-12.31841,-9.63197
+-9.76100,8.09306
+11.92260,9.76728
+9.05025,-9.21890
+-10.37869,-10.24626
+-10.64581,10.53644
+10.31411,11.47423
+9.12264,-11.78265
+-10.61015,-7.00206
+-10.48259,10.17257
+9.90274,11.40645
+9.41668,-9.61940
+-11.04491,-11.16837
+-9.38447,9.11796
+10.15147,9.44395
+11.25667,-11.33330
+-11.19570,-10.80670
+-8.67451,9.59610
+10.20096,8.18139
+8.76359,-9.53850
+-8.51170,-10.82474
+-10.80444,10.43022
+11.08791,10.58632
+10.37907,-9.61700
+-8.49403,-9.99362
+-9.32094,11.86401
+11.58498,10.51311
+11.40479,-10.73107
+-9.14023,-11.15871
+-11.91089,10.50103
+7.37367,11.26768
+11.65241,-9.24869
+-8.04208,-10.59437
+-10.74196,10.59819
+9.90713,9.63949
+10.57509,-10.89718
+-11.21506,-10.83243
+-9.48534,11.75429
+9.58954,9.01128
+10.13821,-11.31203
+-9.66993,-12.04116
+-9.82336,10.21871
+9.08933,10.69874
+11.25360,-8.34728
+-9.27628,-8.81252
+-9.07641,10.45882
+10.17039,10.48503
+8.80340,-9.42473
+-9.56696,-11.47140
+-9.34617,11.60878
+8.77206,10.24910
+9.82619,-8.74768
+-11.42922,-9.74339
+-10.21401,7.98200
+8.22263,9.94200
+9.47764,-10.69919
+-10.99466,-8.46968
+-10.51799,9.37889
+8.35639,9.48361
+9.91017,-10.97399
+-10.19053,-11.02014
+-10.25302,11.06888
+10.33289,10.59716
+10.05981,-11.55889
+-9.25410,-9.58238
+-9.84499,10.35590
+9.77503,9.73675
+10.60765,-9.17312
+-7.90006,-10.54116
+-9.49866,8.15362
+8.63713,10.45885
+9.58624,-10.43803
+-9.82243,-10.06619
+-10.41152,9.59727
+10.11313,9.96882
+11.62735,-10.26748
+-9.68319,-10.35671
+-11.56471,9.48536
+11.01637,11.15864
+9.53896,-11.78939
+-10.80993,-10.99877
+-7.46877,9.73581
+8.79577,10.30503
+9.14628,-10.16503
+-8.00884,-9.64176
+-11.24801,11.56737
+11.39722,10.20053
+10.66053,-10.58677
+-8.69669,-10.26283
+-11.30020,9.81889
+9.10139,10.67074
+10.71595,-9.18575
+-10.15561,-10.15318
+-10.23937,8.65492
+11.24417,10.37165
+11.14602,-9.83730
+-9.41092,-10.51395
+-10.08880,8.14358
+10.61182,9.84015
+9.27557,-10.49778
+-10.59943,-11.14877
+-11.79125,9.54734
+10.75497,9.86990
+8.42073,-9.61616
+-11.39234,-10.17387
+-10.51984,10.71811
+10.15653,10.06226
+10.43647,-11.27505
+-11.42953,-9.86455
+-9.31901,11.80858
+10.60657,8.25900
+9.47332,-12.45148
+-8.68859,-11.45334
+-12.28573,9.64884
+9.72301,10.34778
+10.24780,-10.81403
+-10.07418,-10.85679
+-10.70514,12.08935
+10.41262,11.50164
+11.28683,-7.28880
+-10.19694,-10.17570
+-9.93457,11.21499
+9.67499,10.55983
+9.74621,-10.69505
+-10.04893,-10.82366
+-9.48989,9.49901
+9.60403,8.73688
+10.24126,-11.85583
+-10.01761,-10.26167
+-10.51325,10.71126
+9.35975,9.82274
+9.83578,-10.44581
+-9.56309,-6.52828
+-9.74433,8.94644
+8.36408,10.12437
+7.44821,-10.35626
+-11.40572,-9.54949
+-9.11401,12.03801
+9.65420,9.12896
+11.06282,-10.07780
+-10.16919,-7.99441
+-12.54065,10.50997
+9.06724,8.78590
+10.72754,-9.70419
+-9.04429,-8.77516
+-9.70757,11.18482
+13.48129,10.12361
+8.35404,-8.99100
+-9.08935,-10.19932
+-9.61282,11.41324
+10.83898,8.47669
+9.46155,-10.31281
+-8.51050,-9.74114
+-11.17562,9.14276
+9.34366,8.58650
+10.33750,-9.75658
+-10.11656,-8.08544
+-9.55324,9.64979
+9.93351,9.43296
+10.38098,-11.14134
+-12.15223,-8.10244
+-9.09899,11.21853
+9.98736,8.88194
+8.93314,-10.05870
+-11.92759,-11.68925
+-10.35046,9.56470
+10.20499,8.78451
+9.16889,-10.95730
+-9.67902,-9.81141
+-11.01073,9.70664
+9.82620,9.96091
+10.76079,-11.73664
+-10.49564,-9.60215
+-10.15691,9.94197
+10.42190,11.26826
+8.69291,-10.50335
+-11.62687,-9.64158
+-10.54092,8.21473
+9.52851,9.91478
+9.75550,-10.04142
+-9.22827,-10.27931
+-9.70945,10.07019
+10.23454,9.49210
+10.74247,-9.16856
+-8.19415,-10.29276
+-11.80942,9.15190
+9.40567,11.45762
+10.16863,-10.33713
+-10.28033,-10.76378
+-11.24930,8.68236
+8.01434,8.87712
+10.75045,-10.76150
+-9.63630,-8.99772
+-8.52691,10.27389
+7.56291,9.81482
+9.45605,-10.16712
+-10.95324,-10.45114
+-9.53055,9.19706
+10.01974,10.23586
+8.91488,-7.68245
+-8.23411,-7.72816
+-9.27315,9.65110
+9.00974,10.69162
+9.49728,-8.89390
+-10.73803,-10.91021
+-10.50771,10.38260
+10.29355,10.02205
+11.82024,-9.81528
+-9.64592,-8.98080
+-9.06750,9.16003
+9.63582,10.06773
+10.55754,-9.45372
+-8.54152,-9.25520
+-9.60038,9.69343
+9.96224,9.74533
+10.30236,-10.10761
+-9.93745,-10.82234
+-8.99174,11.61640
+9.18478,10.31262
+10.25776,-11.50917
+-9.96509,-11.05187
+-11.55791,9.69478
+9.57560,9.38101
+9.69312,-10.18378
+-10.77035,-9.75534
+-9.78750,11.13989
+9.66691,10.25202
+10.51754,-10.77718
+-8.97985,-8.77709
+-9.31579,9.45772
+8.35869,10.68560
+8.63272,-10.52797
+-10.86589,-9.23911
+-8.58586,10.62711
+10.67528,11.48796
+10.53145,-9.73381
+-11.12800,-9.20937
+-10.65885,10.00469
+10.20753,10.74392
+9.00432,-8.20518
+-8.90207,-9.58293
+-9.13057,10.13979
+9.58313,10.27831
+9.74227,-11.05970
+-11.36613,-8.58904
+-10.44277,9.12126
+9.38028,8.52496
+9.76164,-11.14644
+-9.39793,-10.69113
+-9.64210,10.74927
+9.53730,10.13722
+9.88495,-10.48610
+-8.84051,-10.09097
+-8.59817,9.66584
+8.73270,11.02126
+11.56391,-9.00984
+-8.91918,-10.44666
+-9.98141,9.38795
+10.25988,9.37857
+10.69807,-8.56477
+-9.62495,-10.72258
+-10.40168,10.35820
+8.98223,10.50026
+10.60603,-10.85991
+-11.52623,-9.46163
+-10.64229,9.60838
+10.84474,8.91006
+10.36052,-10.82490
+-9.23309,-11.13923
+-8.92055,9.46700
+11.45600,8.38241
+7.43671,-9.23170
+-12.49882,-9.93735
+-10.84252,10.45570
+8.84888,10.31108
+10.95107,-10.52766
+-10.44212,-9.57502
+-10.60006,8.61064
+10.92234,10.62414
+9.53466,-9.75173
+-10.65310,-9.83692
+-10.06974,9.34340
+9.15718,9.18972
+10.91118,-10.38710
+-8.10548,-9.74628
+-9.83679,9.94296
+10.12104,7.89510
+10.32603,-11.02276
+-10.69770,-9.87788
+-12.26283,10.56805
+11.64849,9.14539
+10.90884,-11.59009
+-9.44463,-8.60594
+-10.56363,8.09749
+10.09202,12.12250
+10.64831,-10.62360
+-10.02114,-9.39020
+-9.89534,9.50723
+10.30336,10.77142
+10.65073,-9.48321
+-10.45186,-10.37502
+-10.66864,9.53551
+10.57945,9.02810
+10.06204,-10.27175
+-8.72695,-10.09767
+-11.86014,10.27476
+9.66441,8.51046
+9.80629,-11.34922
+-10.09190,-9.84361
+-9.76242,12.60101
+9.35482,9.00474
+11.25846,-10.73255
+-11.01339,-10.39299
+-9.68866,9.46762
+10.52849,10.37741
+10.26659,-9.74206
+-11.33444,-10.42828
+-9.53689,9.39460
+11.13652,9.90804
+9.56845,-8.80177
+-9.55039,-11.46087
+-10.84201,9.72532
+8.51724,10.76520
+7.59388,-10.35139
+-8.15555,-8.89387
+-10.72292,10.45563
+9.82499,10.24415
+9.34912,-10.56456
+-9.30776,-9.03199
+-10.66392,10.11357
+10.74930,11.59970
+10.58539,-12.19491
+-10.73668,-10.06639
+-10.25270,10.08444
+9.73121,11.43765
+11.11215,-10.72316
+-8.99801,-9.03227
+-9.16404,9.11523
+8.85370,8.56141
+10.14416,-10.41651
+-10.59675,-9.03786
+-9.37074,9.70614
+10.47206,9.42690
+8.88755,-10.76625
+-10.38327,-9.27704
+-10.62821,10.31574
+10.81882,9.52330
+10.16052,-12.65426
+-10.58536,-8.96912
+-9.42234,11.54412
+9.46059,10.39359
+10.88396,-9.38772
+-11.31656,-11.12283
+-8.95122,10.69908
+8.81555,8.42139
+10.25158,-10.49006
+-10.05285,-9.06613
+-10.67152,10.61346
+8.77454,9.33660
+10.21278,-10.53929
+-10.05821,-9.62044
+-10.11169,9.88152
+10.44274,9.51997
+12.14476,-10.41239
+-9.75349,-11.79478
+-10.54999,9.29220
+10.75266,10.25113
+8.56790,-8.36213
+-9.82695,-11.24695
+-10.12206,9.39854
+9.86021,10.03411
+9.51931,-9.94323
+-11.09059,-9.51598
+-11.07364,8.61763
+10.66551,9.92426
+11.37123,-9.90062
+-9.65415,-9.07316
+-9.96741,10.32132
+10.16629,9.36910
+10.51544,-9.45630
+-11.25484,-8.80224
+-9.48449,11.23382
+10.18074,10.71542
+9.48369,-9.87914
+-10.55713,-9.25071
+-10.16896,9.69067
+11.08224,11.57497
+8.35463,-10.01276
+-11.75171,-9.77124
+-9.76020,10.15036
+11.64344,11.17697
+9.64570,-8.40446
+-8.98336,-12.12951
+-10.39044,11.88491
+9.96380,9.44632
+10.80557,-10.73396
+-9.54216,-9.80587
+-10.23568,9.29298
+10.08661,8.57125
+9.20444,-10.89618
+-10.85375,-9.78121
+-10.63508,10.01527
+12.44845,12.13809
+7.87586,-10.14511
+-8.70507,-10.25930
+-9.52986,12.10810
+10.89891,10.22408
+11.66414,-9.71210
+-9.75106,-8.99861
+-9.78326,9.52853
+8.78879,9.99032
+11.35070,-9.80115
+-9.12237,-9.48884
+-10.74961,10.90894
+11.34551,10.04033
+9.34883,-9.08387
+-9.77420,-9.59041
+-9.13847,9.23663
+10.26317,9.71817
+11.57047,-12.56156
+-10.61770,-11.15220
+-11.41617,7.92708
+11.81354,10.77361
+10.44117,-11.80012
+-10.83444,-10.16348
+-10.33307,10.06530
+8.53835,10.08073
+9.49381,-8.90539
+-8.60543,-10.29004
+-10.47543,11.06410
+9.51147,10.64312
+9.36126,-7.25349
+-9.82924,-10.04305
+-10.15715,8.85398
+11.04692,9.06828
+11.62515,-10.46257
+-10.09839,-8.60653
+-10.21210,9.58969
+7.72527,10.83738
+10.04813,-9.81621
+-10.24016,-9.42872
+-9.74645,10.15407
+9.70435,10.09397
+11.75579,-11.33653
+-10.63172,-11.10791
+-7.88066,10.89785
+11.11827,10.93411
+10.77483,-11.39468
+-9.39891,-10.01886
+-9.27613,10.74467
+9.57815,11.40842
+10.54789,-8.10484
+-11.79370,-9.12675
+-9.30497,11.46481
+8.25798,9.00323
+9.34031,-8.62301
+-9.33529,-10.03309
+-9.01488,11.57559
+10.44261,10.19745
+11.46342,-9.46015
+-8.90510,-10.34636
+-8.95184,10.19523
+9.37166,8.34478
+9.58485,-10.93833
+-10.44968,-11.60558
+-10.71391,9.46928
+10.97444,10.10256
+10.24466,-9.52279
+-9.75122,-10.31304
+-9.93601,9.82146
+9.96600,10.48373
+10.78756,-10.20365
+-9.85808,-9.74835
+-10.92230,8.19849
+9.62998,10.94383
+10.79267,-9.90174
+-10.59786,-9.92511
+-9.05528,9.06272
+9.73650,10.84980
+11.95568,-10.53520
+-9.82223,-9.91253
+-10.82127,10.94662
+10.95983,10.72550
+8.76338,-10.26160
+-9.84804,-9.63022
+-11.21288,11.68501
+9.96288,8.78077
+10.65233,-9.61691
+-10.27605,-10.53760
+-9.36934,9.26163
+10.23989,10.69327
+10.88802,-11.61623
+-10.59108,-11.71112
+-11.38054,9.93251
+9.79872,12.42502
+10.01785,-10.01953
+-9.97153,-10.44840
+-10.24558,11.15220
+10.54335,11.20850
+10.19341,-11.20758
+-9.40619,-10.71665
+-9.45244,10.76240
+11.11344,9.21985
+10.27932,-9.31139
+-8.91772,-9.59601
+-8.66276,9.57290
+8.92683,9.94794
+10.28346,-10.64379
+-10.24217,-8.79084
+-9.11484,10.24459
+10.21392,9.87376
+10.74770,-10.59908
+-10.53143,-9.50050
+-9.12740,8.12905
+9.18783,10.33268
+10.22430,-9.04274
+-9.70203,-10.55442
+-11.25846,9.52034
+9.89167,10.28292
+8.92355,-11.60056
+-9.95793,-9.86921
+-10.24732,9.85692
+7.75269,10.20571
+10.78048,-9.69522
+-10.49731,-9.04992
+-10.02587,10.84024
+9.90277,9.73029
+9.50367,-10.20458
+-8.86705,-9.38998
+-11.51745,9.65029
+9.62501,10.49536
+8.21793,-9.06838
+-11.11872,-10.55460
+-10.73256,10.17740
+8.90817,9.97583
+9.71323,-9.80359
+-8.03777,-10.17143
+-10.72670,11.92973
+10.71413,11.07704
+9.89133,-8.34466
+-9.97110,-10.49560
+-11.23419,9.07974
+11.24464,7.56567
+11.70997,-10.91340
+-8.66987,-10.70628
+-9.95540,10.96898
+11.79129,7.67136
+9.32494,-10.15996
+-10.13084,-10.31802
+-7.56856,9.81921
+9.39776,10.25347
+8.14788,-8.86602
+-10.48547,-11.15249
+-8.25427,11.68461
+9.04681,8.00131
+10.40208,-10.37708
+-9.75600,-9.82908
+-10.28874,9.51060
+10.56070,12.25900
+11.12561,-9.69392
+-9.85793,-10.41518
+-10.54663,9.33150
+9.68798,9.40530
+10.70968,-8.95781
+-7.78028,-8.18974
+-10.18515,10.01960
+9.66021,10.30358
+10.04503,-10.39082
+-9.68131,-8.76210
+-8.29136,9.47761
+9.88898,9.90563
+9.17131,-11.05595
+-9.93290,-9.34027
+-8.73137,9.43320
+8.52900,8.54686
+10.08903,-10.45040
+-10.09466,-9.94978
+-10.72905,11.10974
+9.86910,10.09491
+10.06967,-8.52981
+-10.12939,-11.58312
+-10.58728,10.95683
+11.62910,10.74365
+9.54654,-9.72203
+-7.56954,-12.13112
+-11.67031,9.11145
+11.04114,9.60654
+10.37341,-9.93573
+-10.78427,-10.96443
+-9.45544,9.90235
+11.37348,11.30294
+9.78640,-10.43096
+-8.97819,-10.16100
+-10.25073,9.34644
+10.24163,11.61851
+10.29623,-10.11524
+-9.97986,-12.28686
+-7.81695,10.39333
+10.22455,9.04144
+9.49091,-9.84904
+-10.73213,-9.83650
+-9.66003,8.46915
+11.14572,10.76209
+11.94042,-10.66189
+-11.15253,-8.86912
+-10.35489,10.38945
+11.15856,10.49696
+10.62192,-9.36666
+-8.79639,-10.81455
+-9.26423,8.87565
+11.45197,7.49338
+10.66006,-11.22723
+-11.02552,-10.05416
+-9.49340,9.83443
+10.61841,9.91074
+7.85202,-9.94662
+-9.98686,-9.06896
+-9.56691,8.66694
+10.80778,11.24957
+9.77054,-10.67540
+-8.77274,-9.24351
+-10.89937,8.95184
+10.29755,11.73105
+8.50874,-10.21897
+-9.16383,-11.84577
+-10.68954,10.01391
+10.41942,10.36529
+10.62471,-8.99301
+-10.26496,-10.03866
+-11.11973,10.37786
+10.22445,10.14548
+10.69951,-10.51622
+-8.72017,-9.20717
+-11.29254,8.86942
+9.32438,7.98722
+10.78590,-11.11507
+-9.89396,-9.19382
+-9.42145,11.15360
+8.88457,9.96513
+10.85122,-12.05737
+-10.30618,-10.13981
+-8.44967,9.27077
+11.04309,10.66595
+9.21450,-11.19105
+-12.26002,-8.68803
+-10.01220,9.98165
+9.98635,8.73453
+9.22729,-9.11203
+-9.67046,-10.52826
+-10.12864,8.72748
+9.25832,11.19309
+10.12099,-9.82747
+-11.09190,-10.01234
+-10.05614,9.56152
+7.39031,11.93526
+10.64419,-9.46459
+-11.91446,-9.89528
+-10.72570,8.93352
+9.59106,9.28769
+11.55461,-10.17316
+-10.61432,-9.00395
+-8.76464,8.35004
+9.41892,10.92608
+11.95788,-9.75861
+-10.48143,-10.26787
+-10.32826,11.13345
+8.91696,8.69185
+9.22754,-8.57563
+-10.95469,-8.47188
+-8.26513,10.67082
+8.81349,10.08452
+10.59407,-10.36260
+-10.81193,-11.63575
+-9.12747,10.29331
+12.53737,9.60405
+8.96308,-10.34562
+-9.52005,-9.48293
+-7.58658,8.46488
+10.89066,9.88026
+9.63928,-8.54420
+-9.29401,-11.11209
+-8.63907,9.57108
+10.43450,10.69783
+9.55953,-9.81925
+-10.75902,-8.19658
+-10.69228,8.75898
+11.28320,10.36823
+10.79127,-8.22910
+-9.78566,-10.70296
+-11.11118,10.93999
+8.68802,9.87376
+10.18619,-10.32766
+-8.41137,-9.10801
+-10.66493,10.94651
+9.61266,10.32781
+10.87980,-9.27175
+-8.71683,-9.29631
+-9.09196,9.99351
+9.76614,12.14974
+11.62538,-8.94080
+-9.05294,-10.07798
+-9.67082,8.27783
+10.24267,11.30997
+10.89141,-9.97704
+-11.48824,-9.77327
+-11.30762,10.21616
+10.79670,10.52483
+9.25644,-7.38763
+-9.61917,-9.52508
+-11.37692,11.21986
+10.01860,10.09102
+9.83624,-11.91557
+-11.37131,-10.16863
+-9.98545,10.80293
+9.15938,9.93694
+10.11785,-10.95669
+-11.92362,-11.43768
+-10.02067,11.99624
+9.26329,9.19020
+10.91991,-9.88019
+-10.98117,-11.92340
+-9.13135,9.13344
+9.60965,10.88259
+9.11799,-10.39832
+-10.05129,-10.26899
+-11.54250,9.76850
+9.35155,10.97164
+11.81784,-11.32478
+-9.81289,-8.93170
+-10.55539,10.33026
+9.45659,8.84852
+10.32836,-10.72945
+-10.23178,-10.25695
+-9.64909,11.36850
+10.18110,8.44790
+10.55511,-9.90677
+-8.61295,-11.63726
+-9.43402,8.56345
+10.45344,10.23905
+10.25613,-10.70437
+-10.25915,-10.22104
+-10.40463,9.08582
+10.98912,9.33005
+11.76766,-9.71113
+-8.60697,-12.72164
+-10.45062,7.79131
+11.29647,9.56541
+10.58536,-9.72502
+-11.14039,-10.42496
+-9.12965,9.29342
+8.74383,11.46429
+8.20653,-8.16070
+-8.61476,-10.33441
+-10.12355,10.21543
+10.11748,8.20202
+11.52560,-9.72237
+-10.23465,-10.05087
+-11.15905,10.41008
+10.14369,7.97801
+9.71966,-11.56886
+-8.95692,-9.24006
+-9.37192,8.72946
+9.52935,9.88453
+10.39296,-11.50370
+-10.44704,-9.13750
+-10.62363,8.93162
+9.53509,10.17973
+9.73352,-10.51968
+-9.68964,-9.58258
+-9.87390,10.43695
+10.81567,10.67781
+10.58838,-9.22601
+-10.38423,-8.98281
+-9.70363,10.62299
+8.82821,8.28171
+9.85509,-10.58284
+-8.11897,-8.77446
+-10.39552,9.50391
+7.34570,8.80573
+8.29212,-9.97446
+-11.56693,-11.04367
+-8.40089,10.11475
+11.31671,11.27293
+10.54995,-8.69093
+-9.19051,-10.70357
+-9.20358,11.38065
+10.09525,10.41315
+10.30405,-9.06508
+-9.12263,-10.99786
+-10.20798,10.97353
+9.99449,11.12323
+10.15216,-9.49739
+-9.18715,-9.99176
+-9.11645,9.29499
+9.04466,10.02004
+10.45619,-8.12302
+-11.00923,-9.20516
+-8.46219,9.65200
+9.99733,9.65520
+7.97438,-9.16282
+-8.93666,-9.21697
+-9.24245,10.04087
+9.48412,9.92366
+9.35702,-11.23201
+-10.97453,-8.75181
+-9.88658,9.26743
+11.21228,10.46175
+12.73656,-9.99046
+-9.49962,-9.41785
+-9.43322,10.07447
+9.99034,9.53410
+9.02283,-9.41931
+-9.99082,-9.89778
+-9.90292,10.64367
+12.03917,10.02233
+8.09723,-8.74109
+-10.31176,-9.74893
+-11.00971,8.75993
+10.51291,8.91803
+9.95666,-10.74894
+-9.01952,-12.20648
+-9.66542,11.76197
+10.13619,11.49557
+8.55364,-10.14087
+-9.75054,-9.99049
+-10.17310,10.27943
+8.62207,9.43856
+12.54607,-11.71364
+-10.13205,-8.62289
+-9.65807,11.45872
+10.00376,9.06094
+11.41486,-9.49884
+-9.74633,-9.72682
+-10.69946,8.63986
+9.31303,10.09145
+9.13606,-9.24155
+-10.76424,-9.59693
+-7.49077,10.42767
+12.52354,11.79381
+10.84121,-9.72757
+-9.49894,-10.21498
+-10.56189,10.45259
+9.84828,9.53039
+9.35265,-10.38720
+-9.11318,-10.31222
+-9.26640,9.49044
+9.53359,10.28208
+11.50709,-10.24385
+-7.84589,-12.01852
+-9.00808,10.29170
+9.34953,10.59889
+11.65568,-10.61631
+-10.12573,-9.49916
+-10.05013,10.52164
+10.75531,10.49859
+9.63544,-9.80345
+-11.56952,-10.14217
+-7.78857,8.99261
+9.25459,9.57805
+9.38733,-9.51183
+-9.49077,-11.30215
+-9.53314,9.78173
+9.66933,10.62553
+10.62903,-8.97003
+-10.33323,-8.77702
+-8.49686,9.38104
+10.43120,9.93107
+9.70964,-9.72325
+-10.13464,-11.37267
+-10.11537,9.61495
+10.91690,8.93711
+12.30947,-12.11075
+-12.50909,-9.47976
+-10.52416,9.65160
+11.28972,10.73509
+9.47303,-10.89848
+-10.08815,-10.45039
+-11.15108,11.43755
+9.80637,10.62650
+11.72626,-10.54226
+-9.37331,-9.66255
+-10.46800,10.35433
+11.15755,9.13192
+9.90877,-10.01262
+-8.67620,-9.30940
+-10.77624,12.28732
+11.04259,10.06663
+8.53273,-11.23488
+-11.00788,-10.74200
+-10.21459,10.21009
+11.77647,7.74259
+9.39824,-8.64369
+-10.49665,-8.77616
+-8.32895,11.29464
+10.25214,9.70447
+8.96591,-11.34058
+-9.27474,-9.79907
+-9.88810,9.30395
+10.79392,9.73186
+8.85181,-8.92021
+-10.34230,-10.93326
+-9.93047,11.07439
+11.05409,8.83117
+9.69781,-10.36674
+-9.13126,-9.74775
+-9.30526,8.54333
+10.35554,9.61942
+10.61007,-11.74438
+-8.65108,-9.67894
+-9.34914,10.05951
+10.83648,10.91053
+9.98526,-10.35408
+-10.85288,-8.20265
+-9.05587,10.34205
+11.39520,10.50432
+8.58613,-7.85556
+-9.95695,-10.84676
+-11.31807,9.65262
+8.67629,10.53755
+8.44477,-11.66147
+-10.73848,-10.67859
+-11.42377,8.83271
+10.74657,11.07026
+10.00902,-9.36497
+-9.84043,-10.24067
+-8.64072,9.19804
+11.46755,12.13156
+10.44621,-10.02979
+-8.26852,-9.45340
+-11.97868,11.73534
+10.82204,10.13795
+9.62638,-10.70124
+-8.97094,-10.38996
+-10.80134,9.25373
+10.72506,10.01328
+9.39014,-11.11078
+-9.53424,-9.96255
+-8.57368,9.64903
+9.53224,10.57839
+12.20055,-10.10248
+-9.82695,-10.18075
+-10.43363,9.62846
+9.85892,9.88243
+10.88019,-9.33378
+-10.26621,-10.22795
+-10.69273,9.43482
+10.50152,9.99446
+10.14174,-9.27924
+-10.22033,-9.81932
+-10.07106,11.46228
+9.71150,9.54903
+11.40553,-11.47013
+-8.32078,-10.61338
+-10.56354,9.76339
+8.34103,10.00271
+9.23987,-9.97100
+-10.05031,-10.39936
+-11.08764,10.59772
+8.90167,10.57172
+7.98550,-13.06883
+-10.79018,-10.02269
+-10.26400,10.25406
+9.72095,9.16560
+10.04722,-9.76480
+-8.59400,-11.42453
+-9.75870,10.11094
+11.01704,9.23467
+8.39860,-9.72912
+-10.34087,-9.92246
+-9.80698,10.83824
+10.45041,9.46318
+10.23929,-9.78308
+-10.63981,-9.37525
+-9.71432,10.15701
+7.67558,11.89249
+9.55111,-9.90697
+-9.71252,-10.32087
+-9.93606,10.69155
+8.11134,9.98827
+8.14103,-12.28892
+-10.47241,-8.50301
+-9.14181,10.14153
+9.39603,10.71483
+10.72396,-10.21389
+-9.28112,-10.81025
+-8.47287,9.72466
+9.90486,9.99691
+11.76694,-10.87478
+-8.16279,-10.43223
+-9.92171,8.66702
+8.81190,11.02314
+12.09701,-8.92954
+-11.90860,-9.65615
+-8.85843,8.61905
+11.49314,11.06578
+12.00923,-9.98417
+-8.71367,-10.78017
+-10.07296,10.33561
+9.14445,10.97266
+10.59677,-9.89197
+-10.10791,-8.28347
+-10.27069,8.60752
+8.16969,12.93739
+8.87957,-10.93849
+-10.28498,-10.27073
+-8.10026,9.87371
+10.04321,8.13961
+9.31668,-9.19594
+-8.31791,-7.96978
+-5.95184,10.33896
+10.21077,10.32751
+10.02866,-10.56737
+-10.11670,-9.15646
+-10.10267,11.07642
+10.24617,8.81674
+12.47122,-9.71179
+-9.45373,-9.84185
+-8.41201,9.27622
+9.67310,11.95515
+10.92683,-10.24379
+-9.86547,-10.05519
+-11.39350,9.86916
+9.35974,10.36083
+9.27165,-10.75090
+-10.96182,-9.53067
+-9.27077,9.45328
+10.31649,9.99871
+11.06503,-10.58772
+-9.10752,-9.56885
+-11.06017,9.63588
+11.47064,10.50389
+9.06744,-10.26751
+-10.29384,-10.75059
+-8.99075,7.86529
+12.64365,10.62619
+12.08994,-10.27460
+-9.13678,-9.81591
+-9.57999,10.55324
+10.30067,9.73879
+11.50785,-10.36675
+-11.81235,-9.89639
+-9.06783,10.08295
+12.17723,10.40391
+10.39822,-11.21377
+-10.46707,-9.65256
+-10.97826,8.44082
+10.60501,11.10233
+11.10488,-11.66117
+-9.84944,-9.38299
+-10.69577,11.08181
+9.94755,8.60324
+9.77645,-10.12030
+-9.62704,-9.92030
+-10.84507,10.34257
+8.85391,9.89764
+9.14725,-9.84328
+-8.92362,-9.23577
+-9.87055,10.50879
+10.02137,9.71290
+8.73020,-10.07993
+-9.53584,-10.03129
+-10.30800,8.96887
+9.85227,10.20948
+9.97215,-9.46434
+-10.27616,-10.94801
+-11.04242,12.15702
+9.38614,11.52291
+11.49339,-10.74395
+-9.24136,-10.35170
+-8.78925,9.88953
+12.92539,9.71651
+10.67657,-9.95942
+-9.87779,-11.39505
+-9.90493,10.53326
+9.43082,9.80016
+9.28528,-11.18237
+-8.88846,-8.71323
+-9.26453,11.27078
+9.54350,9.54994
+9.38963,-8.39350
+-9.13468,-9.70792
+-10.33250,10.58732
+12.79913,11.68292
+9.55385,-9.60144
+-10.04093,-9.49521
+-9.24599,10.91334
+12.13366,11.28025
+11.21634,-9.38340
+-9.50632,-11.46780
+-8.46096,10.24495
+9.59943,9.79970
+9.01960,-10.60707
+-9.81269,-9.94558
+-9.50161,9.84415
+10.50773,7.44610
+8.71142,-9.33368
+-10.80124,-10.50482
+-10.48570,9.27164
+7.87341,11.44458
+7.85319,-10.57173
+-8.06536,-8.55813
+-10.56072,10.66037
+9.70337,9.43205
+9.26535,-8.98410
+-9.43501,-11.03626
+-10.74544,11.38557
+11.15442,9.35422
+10.30845,-10.62539
+-10.58942,-10.01472
+-9.08633,11.62972
+7.11941,9.27040
+9.84195,-11.12936
+-10.69463,-10.47691
+-8.80749,11.15291
+9.51496,11.28999
+10.46749,-10.08125
+-9.93181,-10.12608
+-9.75412,11.17023
+8.41173,10.56237
+9.62874,-10.25282
+-12.52625,-8.96287
+-10.29346,9.21429
+10.35243,10.25237
+10.11556,-10.83158
+-8.19607,-10.04876
+-9.97137,9.54959
+9.12169,8.15008
+9.39866,-8.21460
+-11.78750,-8.94098
+-10.36043,9.30429
+10.40067,11.88656
+8.89566,-10.61740
+-8.90183,-11.57705
+-9.75815,11.01465
+8.86092,10.22906
+9.77027,-8.30091
+-10.33466,-10.63339
+-9.86187,9.70739
+8.62760,9.58358
+10.81370,-8.31706
+-11.24269,-8.46120
+-10.54923,13.05221
+8.70009,11.02341
+9.14959,-9.06785
+-9.48365,-8.90846
+-9.98579,10.55339
+10.84823,10.74101
+10.80006,-12.06440
+-8.58982,-9.48311
+-9.21710,10.99731
+8.88786,11.83802
+10.76721,-9.91080
+-11.84052,-9.97958
+-9.81901,10.29073
+11.63452,8.57482
+10.83476,-10.39228
+-10.21513,-10.98692
+-11.22096,9.45495
+9.66937,9.99933
+9.21847,-11.15737
+-10.57447,-9.93272
+-10.35595,9.61334
+9.76212,10.72538
+9.59118,-11.97270
+-9.40190,-9.31472
+-11.93822,11.56589
+6.81798,8.54633
+8.26942,-10.63299
+-11.69434,-8.24359
+-9.70991,12.51519
+11.52609,10.24201
+10.82730,-10.24331
+-10.94760,-9.71734
+-10.45774,9.96225
+10.50692,10.57767
+10.43143,-9.88913
+-9.78782,-9.71804
+-10.38203,7.36567
+10.00414,10.60148
+10.11765,-10.02620
+-9.73063,-10.20114
+-7.18649,7.58676
+10.22319,10.31563
+7.85226,-10.94440
+-11.11321,-10.85444
+-8.92124,10.03305
+10.48778,10.50467
+8.81298,-11.48811
+-11.04864,-10.58717
+-9.72597,9.68721
+10.86041,10.16032
+9.91105,-9.83334
+-9.24230,-10.05373
+-7.41277,10.38944
+9.70169,8.94774
+9.81348,-10.63826
+-10.39206,-11.41955
+-10.95379,9.51668
+10.25788,9.73281
+9.73392,-8.62686
+-9.00386,-10.74695
+-9.82144,8.90011
+9.98497,9.04926
+10.39934,-11.01044
+-7.62911,-10.25456
+-9.27625,10.20989
+10.92335,10.50408
+9.10518,-9.63283
+-10.38299,-10.05451
+-10.32787,9.58995
+10.90783,9.34246
+10.28446,-9.92625
+-10.64367,-9.10678
+-12.24911,9.36533
+10.06206,9.28431
+8.72459,-9.97665
+-11.14908,-10.50844
+-9.43403,10.31132
+10.25602,11.07399
+10.50287,-9.62650
+-10.28843,-10.55488
+-12.38304,10.54503
+9.72923,9.13533
+9.34427,-8.61904
+-12.38771,-8.16824
+-10.64579,9.11632
+9.10088,9.46637
+10.65552,-11.18283
+-9.93548,-9.82281
+-9.61234,8.94174
+9.15252,11.65705
+9.54119,-9.15152
+-8.38536,-11.56255
+-12.12048,9.86712
+8.44830,9.93645
+11.43989,-11.78953
+-9.58830,-9.82403
+-9.21746,9.52309
+10.02274,9.59183
+9.17708,-11.96979
+-10.05198,-9.26635
+-9.32263,9.98534
+10.33721,10.34637
+8.43543,-13.51652
+-8.93277,-9.11332
+-8.56813,9.88001
+9.74861,10.19968
+9.91098,-10.33689
+-11.15261,-10.99046
+-9.66165,7.69591
+10.95919,9.63387
+8.81154,-10.76357
+-10.50308,-10.59360
+-10.11814,10.20607
+11.05775,10.79161
+10.94759,-9.70960
+-9.02709,-8.67873
+-9.54063,9.19228
+10.31093,11.07549
+9.03759,-10.07469
+-9.67075,-9.84911
+-10.54492,10.01688
+8.30845,11.07977
+10.61868,-10.66914
+-8.67764,-9.81903
+-9.69288,9.39900
+10.75793,10.81828
+9.31805,-8.78887
+-8.87647,-10.34200
+-8.36224,10.08076
+9.01979,11.51651
+11.67443,-10.22496
+-11.33785,-8.64476
+-10.80503,9.35342
+8.54347,8.83635
+9.07960,-8.14325
+-9.90798,-10.29000
+-9.12141,11.00704
+11.34343,8.99587
+10.47660,-11.48639
+-10.23304,-8.80061
+-9.60933,10.00627
+9.97620,8.77158
+7.98024,-10.50834
+-9.85957,-9.79331
+-10.77607,11.25083
+9.63346,10.72587
+11.88753,-9.96074
+-11.57714,-9.34381
+-10.05289,10.34829
+11.38804,10.28850
+9.96017,-9.58934
+-10.42843,-11.52483
+-10.61926,7.66291
+10.04661,10.16275
+9.48284,-9.61592
+-8.82138,-8.20020
+-10.84707,11.94133
+8.62614,8.80663
+8.99780,-10.06826
+-7.37381,-9.85104
+-10.23276,10.82482
+9.43535,11.34045
+11.36108,-7.73173
+-8.37744,-11.50940
+-9.77689,10.42606
+11.10050,8.80531
+10.40981,-11.10422
+-9.29833,-11.02953
+-9.66702,11.19286
+11.55114,10.38655
+10.69362,-9.45795
+-8.01913,-7.64431
+-10.16292,10.01593
+8.48190,8.74469
+10.04064,-10.07901
+-9.91683,-11.10758
+-10.19078,9.27520
+11.32502,10.44870
+10.26628,-10.85312
+-9.65059,-8.43930
+-9.75612,10.87197
+10.75029,9.41749
+9.60522,-10.06509
+-9.47807,-9.79733
+-10.53932,8.90913
+9.01431,9.81050
+8.13583,-8.60475
+-8.77932,-9.14935
+-10.14986,12.14375
+10.97756,11.20320
+10.92425,-10.93432
+-8.49921,-11.70657
+-10.59495,10.49045
+10.30811,11.47359
+10.10827,-10.12802
+-9.68621,-10.29412
+-11.25848,10.35997
+9.62611,9.49240
+10.22121,-11.18951
+-11.27836,-9.78620
+-9.42982,10.67091
+9.40776,9.27436
+9.96137,-10.51017
+-9.44684,-10.40816
+-8.64930,9.73565
+8.65862,10.57910
+11.02507,-9.62711
+-11.03915,-9.91754
+-10.93666,10.74104
+9.54207,10.02070
+8.80715,-9.53359
+-11.07949,-11.12115
+-9.57356,9.99076
+9.58557,11.57766
+11.78330,-10.39890
+-9.05279,-9.94243
+-10.25998,11.46537
+9.11575,9.08215
+11.26151,-11.03204
+-9.64922,-9.94467
+-10.23318,9.82387
+10.54071,9.18077
+9.79215,-9.87881
+-8.46183,-9.68594
+-11.60105,9.46402
+11.41822,9.30560
+9.97549,-10.42287
+-10.15895,-10.26671
+-9.14471,10.13291
+9.21949,9.52651
+10.30565,-9.00440
+-10.80897,-9.86827
+-9.23999,9.14688
+8.91337,10.72179
+9.32228,-11.10680
+-10.17443,-9.51773
+-11.00125,10.41572
+9.44738,9.03316
+9.62628,-8.94530
+-12.56359,-9.34104
+-8.83384,8.76160
+9.37865,10.70272
+9.83648,-11.17627
+-9.79116,-10.12735
+-10.21891,11.00847
+9.67845,11.11468
+9.44543,-8.63826
+-10.27022,-10.13472
+-10.28498,10.15917
+10.26059,9.63967
+8.18057,-9.83266
+-9.72511,-11.26491
+-11.63127,9.91704
+8.72387,10.43916
+9.49933,-9.59465
+-9.84114,-8.54283
+-8.68999,10.96824
+9.59738,9.68839
+11.12972,-9.82428
+-8.68703,-12.40799
+-9.44664,9.85329
+10.75556,10.83642
+10.82862,-11.16907
+-7.83501,-10.51394
+-10.12731,10.39974
+9.61571,9.64175
+10.37081,-9.22338
+-10.16243,-9.54857
+-9.84697,9.94376
+12.00229,9.13245
+11.44357,-9.75250
+-10.76768,-9.95349
+-9.42534,10.41958
+10.24718,9.85672
+10.03723,-9.17024
+-10.64853,-11.46557
+-8.10137,11.66450
+9.86509,9.97057
+10.11573,-11.25699
+-9.96607,-10.72960
+-8.31104,9.70436
+10.50228,8.79488
+11.55939,-10.14763
+-9.98500,-9.78766
+-10.90163,11.83167
+9.34562,9.76064
+10.95445,-10.15679
+-10.62430,-9.73190
+-9.96287,9.91559
+11.92288,8.76008
+9.17224,-9.22417
+-11.53495,-10.41287
+-9.55717,10.77358
+10.16861,10.34787
+10.76493,-9.05700
+-12.50452,-10.37502
+-10.53507,9.11117
+10.64691,10.34009
+10.84741,-9.37745
+-10.65134,-9.40479
+-9.95755,9.52778
+10.58696,10.27169
+8.21902,-10.39668
+-8.13278,-9.91265
+-9.99701,8.82157
+10.66897,9.69510
+8.31519,-7.52836
+-10.93135,-9.48438
+-9.36392,9.89198
+11.21111,10.13845
+9.69303,-10.29499
+-9.55202,-9.59103
+-11.66671,9.87087
+9.44135,9.59935
+10.30396,-8.51387
+-10.94534,-9.69078
+-11.61217,9.40321
+10.17757,10.30222
+8.66380,-9.55675
+-10.83737,-10.66558
+-11.79912,8.80209
+9.37586,9.90481
+10.35542,-9.69533
+-9.98701,-10.30918
+-12.40042,9.62964
+10.94085,10.88773
+8.77961,-9.76845
+-9.41680,-9.36185
+-10.72201,10.25853
+9.16739,11.48547
+10.53612,-8.52007
+-9.48117,-9.62966
+-10.30210,9.62621
+11.23561,10.18212
+10.55837,-8.98111
+-10.15223,-12.09389
+-8.46116,9.68774
+11.84983,8.65605
+10.42746,-11.56288
+-8.32144,-10.59575
+-9.16742,10.76177
+10.66689,10.93037
+11.01787,-9.97659
+-10.61587,-8.85687
+-10.46707,9.69446
+8.49131,10.17170
+9.16355,-10.48227
+-7.87074,-10.73678
+-8.86424,9.44481
+9.70223,9.67057
+10.09711,-9.66440
+-12.70748,-8.26034
+-9.80025,7.84185
+10.49146,9.38397
+10.08993,-8.72629
+-8.39093,-10.91054
+-8.12629,8.58853
+9.10402,10.81923
+7.43435,-9.79375
+-10.84159,-9.97187
+-9.91785,11.27835
+10.27463,10.99081
+10.22629,-9.07639
+-9.87162,-9.40094
+-9.92780,10.53101
+8.67729,8.30422
+9.80977,-10.27747
+-9.58220,-8.78005
+-10.45839,10.05185
+9.98054,11.22805
+10.94878,-9.25594
+-8.20881,-11.24219
+-8.80996,7.66415
+11.06902,8.79406
+10.34850,-9.29285
+-9.95422,-9.79399
+-11.33152,8.40830
+9.19124,9.53966
+8.70857,-7.89762
+-9.53700,-8.41474
+-11.06956,10.04225
+9.52618,9.90048
+10.39542,-10.38063
+-10.17282,-9.29451
+-11.30333,9.49468
+8.15966,9.49479
+7.69926,-10.03983
+-10.17268,-10.63234
+-10.67853,10.26892
+10.75952,11.00271
+10.12873,-10.62565
+-9.06285,-7.36671
+-10.91826,7.14162
+8.74013,8.71005
+9.82785,-9.58754
+-10.65004,-9.54038
+-9.92999,10.32063
+10.80162,10.54237
+9.57723,-10.27758
+-9.78587,-10.40575
+-10.87824,10.09292
+10.71590,9.48001
+10.91688,-10.12632
+-9.45334,-8.95882
+-9.00135,10.32459
+10.83092,9.56164
+8.44336,-10.60264
+-9.64594,-10.51990
+-8.66307,9.43606
+8.67239,11.10743
+8.62765,-9.53319
+-9.95340,-10.26363
+-9.53369,9.69529
+9.97610,9.88631
+9.49584,-12.33613
+-12.23096,-11.05449
+-9.50642,10.78280
+10.77182,9.95234
+10.67372,-9.30050
+-10.69057,-10.71085
+-8.17709,10.20564
+9.69902,9.43098
+10.80857,-8.87976
+-9.79988,-8.22082
+-10.54556,8.62053
+10.09638,8.72430
+9.11749,-10.83959
+-9.64764,-9.55555
+-10.83186,8.16097
+8.64895,9.46855
+10.29314,-9.57842
+-9.92877,-11.17250
+-10.80709,8.90396
+10.90270,11.71482
+10.00820,-9.28797
+-10.54193,-11.12007
+-8.94477,10.50677
+9.49604,9.99741
+11.09080,-10.78899
+-11.15660,-9.51568
+-9.14184,10.71654
+9.97800,8.80912
+8.17225,-8.85170
+-10.53641,-11.08221
+-11.78094,11.40384
+9.92480,10.22787
+11.69318,-12.21825
+-11.46057,-7.94364
+-10.66048,9.71977
+8.62237,9.30858
+9.79193,-12.00550
+-9.99473,-11.02022
+-10.14974,8.62882
+11.94829,10.79018
+9.07157,-10.22173
+-8.59741,-10.27867
+-9.45222,10.60786
+10.99186,10.38334
+8.84814,-8.74994
+-8.61634,-10.76515
+-10.76472,12.21515
+9.38339,9.29250
+9.64015,-8.76973
+-10.07585,-8.12549
+-9.03248,12.56559
+11.50265,9.93197
+8.98099,-10.16897
+-10.53767,-10.62512
+-10.46418,9.65279
+11.39266,10.22892
+10.42551,-10.42547
+-10.41459,-11.48316
+-9.12847,8.94929
+11.79427,8.57435
+10.52262,-8.74147
+-11.30681,-9.53687
+-11.91385,10.23614
+10.03178,8.85793
+11.03683,-8.68051
+-11.32907,-10.89253
+-10.11805,9.84243
+10.52627,12.15229
+10.46288,-10.65728
+-9.80097,-10.68963
+-10.13879,9.09979
+8.71142,9.96040
+10.64380,-10.34996
+-8.89388,-9.73863
+-10.22339,9.74778
+10.51307,10.53913
+8.64270,-9.13985
+-9.86910,-10.29630
+-12.14959,10.74648
+8.96625,8.43191
+10.46467,-8.85145
+-10.26828,-10.25732
+-9.37317,10.68068
+10.36741,9.81373
+10.01082,-10.28780
+-10.57113,-9.86823
+-9.12237,9.26406
+8.73616,9.99884
+9.70106,-9.73151
+-8.75762,-10.18751
+-9.31931,9.77344
+8.24163,7.95497
+9.95376,-9.02909
+-8.94170,-11.14188
+-10.80487,7.62874
+8.58841,10.34988
+8.64513,-10.90730
+-9.33836,-9.08223
+-11.01586,10.02567
+10.71064,9.77115
+9.77298,-9.62756
+-9.36945,-9.16850
+-10.10546,9.39307
+10.13669,10.15299
+10.12944,-10.82625
+-9.22483,-9.22800
+-10.30847,8.48631
+11.37027,9.67662
+10.72465,-10.64315
+-10.31660,-10.27344
+-9.79080,11.38388
+10.48536,9.71827
+8.92343,-10.06303
+-10.13695,-9.63510
+-11.01703,11.58506
+10.01671,10.77272
+12.10263,-10.85997
+-10.33304,-9.63618
+-8.37303,9.28420
+10.31606,10.66626
+11.01351,-8.14740
+-10.83287,-9.30959
+-9.30831,11.81949
+10.68373,11.14148
+10.84213,-8.20031
+-10.39400,-9.01983
+-9.19228,10.45671
+9.79260,10.76978
+11.56131,-9.44948
+-9.25229,-11.24535
+-10.61533,10.48343
+11.02181,9.57679
+8.26182,-8.59604
+-10.34428,-10.75534
+-8.49153,10.96166
+10.35053,9.26970
+9.91090,-11.17654
+-11.39047,-8.20578
+-10.48177,12.36793
+9.91880,10.05395
+10.63768,-9.44151
+-11.42130,-7.68067
+-8.28822,9.86705
+10.51897,11.20038
+10.43788,-8.83885
+-10.28799,-9.37228
+-10.86415,7.94812
+9.91096,10.24133
+10.70300,-9.52208
+-10.16389,-8.51110
+-11.28797,12.47278
+11.98683,9.83135
+10.48116,-9.78728
+-9.90132,-9.93935
+-9.30313,10.71503
+8.66344,12.25977
+10.32130,-9.95465
+-11.52390,-10.85275
+-10.29215,8.33778
+9.64964,9.83853
+8.36623,-10.19635
+-11.19520,-10.36432
+-10.78806,9.64818
+9.57851,8.89483
+10.20431,-8.67342
+-11.28482,-11.77664
+-9.15856,10.46691
+9.59095,10.91242
+10.37350,-8.40341
+-9.17133,-9.82392
+-10.74754,10.15998
+9.94847,10.48795
+9.69168,-11.77839
+-9.08017,-10.56190
+-10.58582,9.30432
+9.35556,12.06174
+11.02047,-9.68836
+-9.93737,-10.29074
+-9.58649,9.89962
+8.54743,11.15722
+10.46693,-11.53760
+-10.49432,-10.84908
+-9.39805,12.65651
+10.26388,10.51223
+10.66553,-10.16803
+-10.57506,-8.97844
+-9.80032,12.11777
+8.52328,9.34425
+10.90232,-7.86372
+-10.23902,-10.47693
+-9.50758,8.45388
+9.60275,9.14840
+9.76738,-8.31063
+-11.72920,-10.47061
+-9.51819,9.69033
+9.85451,11.66512
+10.12960,-11.21999
+-10.55818,-9.43496
+-10.55908,8.34486
+7.94087,10.10411
+10.73269,-10.22480
+-10.02537,-11.12594
+-10.68004,8.90862
+9.53526,10.10441
+9.95116,-8.49839
+-9.50858,-10.60351
+-9.75175,9.75770
+10.56071,11.24020
+10.18481,-10.26877
+-9.94352,-9.17151
+-9.61021,10.28008
+11.20419,10.71217
+10.20912,-11.29840
+-9.15002,-9.06551
+-7.31733,8.47257
+9.78608,10.55193
+9.29163,-9.91749
+-9.48289,-10.33519
+-10.45926,10.99156
+11.77981,9.72033
+10.02072,-10.86907
+-9.84485,-8.02396
+-11.79440,8.87820
+10.01478,11.03545
+10.36622,-10.70542
+-10.53703,-10.18653
+-9.40400,10.20244
+8.51225,9.66399
+9.91832,-9.41200
+-9.84123,-11.35278
+-10.29698,11.74155
+11.19958,11.21563
+12.81961,-8.88678
+-9.56000,-11.00454
+-9.39900,8.06090
+10.31908,9.89957
+11.25444,-9.91990
+-9.60338,-9.75917
+-11.55091,9.32609
+8.53680,10.62575
+10.03066,-11.91291
+-7.84819,-9.19113
+-8.09588,10.53958
+10.65756,11.32179
+12.20735,-9.08621
+-8.54460,-12.22118
+-8.86313,9.18826
+10.31448,10.72036
+11.13835,-10.39534
+-10.72247,-10.89314
+-9.86520,8.87032
+9.45501,9.92620
+9.85078,-9.12930
+-9.98050,-9.03315
+-10.79505,9.57689
+10.74050,10.59686
+9.67462,-9.19731
+-10.84763,-11.17332
+-10.54760,11.28613
+9.64734,10.15376
+8.84004,-10.74898
+-10.47881,-10.12533
+-10.96754,12.07083
+11.27981,9.38377
+9.63026,-10.02854
+-8.68891,-9.23166
+-11.57726,9.61395
+9.97777,9.37052
+10.55624,-8.75142
+-9.09246,-10.02319
+-10.34547,10.06443
+7.60082,12.01562
+8.71833,-9.00922
+-10.75078,-10.41549
+-9.37076,9.29558
+9.66679,8.19252
+10.01701,-10.10582
+-9.49246,-8.62163
+-9.79780,10.69137
+9.72835,7.86070
+8.89738,-10.84840
+-9.65094,-9.66591
+-11.35683,12.01363
+9.17843,8.77654
+8.33846,-9.20487
+-10.04458,-11.43795
+-9.02336,9.14186
+9.28413,11.13651
+11.60131,-10.13950
+-10.54120,-9.41435
+-9.46194,11.10289
+11.72000,9.88733
+9.05088,-8.90645
+-11.01725,-9.82130
+-11.43956,8.42752
+9.56937,9.50669
+10.32867,-10.92930
+-8.80316,-10.62172
+-10.32309,8.86456
+10.75818,9.03184
+10.56708,-9.22119
+-10.80756,-9.91355
+-8.60080,10.82451
+6.85990,9.53422
+11.26213,-9.97185
+-10.25282,-9.41556
+-8.82677,8.50246
+11.11883,7.67937
+11.22518,-10.55392
+-10.39429,-11.04460
+-9.27964,10.44625
+8.95511,11.37555
+10.22642,-8.95143
+-9.51954,-10.58680
+-11.70534,10.54402
+10.50394,9.90249
+8.97634,-11.05337
+-12.03805,-10.67464
+-11.09362,11.53577
+9.03992,8.73964
+10.49002,-8.93940
+-9.70620,-10.10809
+-9.64218,9.16772
+11.41315,9.15430
+10.81281,-8.73792
+-10.30435,-9.92500
+-10.95706,9.71778
+10.34710,8.96261
+9.81266,-9.97806
+-9.52254,-10.14921
+-11.08428,9.49841
+11.40795,9.87651
+9.71972,-10.18379
+-8.97557,-10.80799
+-9.08726,9.18589
+10.11055,10.61425
+9.97571,-7.15943
+-9.85451,-10.88499
+-11.32217,10.74039
+9.51758,9.36208
+9.82417,-10.38575
+-8.42355,-8.59272
+-10.14024,10.91503
+9.29908,8.92070
+10.46953,-9.12742
+-10.04216,-10.52098
+-10.76414,9.73193
+11.06685,10.85752
+12.13708,-11.04086
+-11.13475,-10.81931
+-10.35736,10.47073
+9.66565,9.52999
+10.27365,-10.24205
+-9.73274,-10.51462
+-9.25596,9.92664
+9.74090,9.97820
+10.43140,-10.41380
+-8.58428,-10.48495
+-10.41484,9.33681
+9.12009,10.34563
+11.04967,-9.35433
+-10.38701,-8.34764
+-9.89170,8.81956
+9.40193,10.08953
+9.43785,-8.72881
+-10.05195,-8.89816
+-10.22222,7.94594
+9.67169,12.01856
+9.90979,-10.27977
+-9.15355,-9.79123
+-12.93783,10.43363
+10.74660,10.23108
+9.14181,-11.68189
+-10.54039,-12.11210
+-10.47021,9.10364
+9.17575,10.57862
+10.63849,-7.48076
+-10.39429,-10.71733
+-9.98119,9.24727
+11.74668,8.29477
+9.17164,-10.83324
+-8.12411,-10.43396
+-10.94448,9.88373
+8.30023,8.65397
+10.22797,-10.42755
+-9.37016,-9.93102
+-11.10094,11.70220
+8.96637,9.87367
+11.73903,-11.46900
+-10.51766,-9.97488
+-9.32606,12.10578
+10.92049,10.12834
+11.35511,-10.47720
+-9.15742,-8.74275
+-10.09903,11.03508
+8.77566,10.35769
+8.93778,-10.52604
+-11.42133,-11.94968
+-9.27794,10.47388
+10.70004,9.46773
+9.47445,-9.95883
+-11.83856,-11.33073
+-8.79978,9.66566
+9.78174,9.06488
+10.36026,-8.86746
+-11.07830,-12.13080
+-8.48637,9.70480
+8.38310,9.67222
+8.90333,-9.07890
+-10.39420,-11.38777
+-8.99206,10.45044
+9.88522,10.03788
+11.22332,-9.39067
+-8.42497,-9.52056
+-9.51513,9.17666
+11.11620,9.97511
+9.92043,-10.40861
+-9.30318,-9.24222
+-9.67638,11.35363
+9.54628,10.49419
+8.65768,-8.26047
+-9.73718,-11.25723
+-10.79347,8.53976
+10.34264,8.59464
+10.33734,-9.85924
+-9.98272,-9.08671
+-10.90493,10.31043
+9.91323,10.12918
+9.16105,-8.80774
+-9.19465,-9.70410
+-11.08514,10.22266
+10.85016,9.58752
+11.01243,-10.26858
+-11.06140,-8.08112
+-9.71101,9.42109
+8.91907,9.26423
+10.34996,-10.76112
+-8.82196,-10.64842
+-9.60132,10.38926
+9.16056,10.69965
+9.42124,-11.19813
+-10.42560,-9.54240
+-9.23772,11.34360
+11.11939,11.15943
+11.64198,-10.24486
+-9.66243,-11.41631
+-9.49509,10.02383
+11.31572,8.49648
+9.05467,-10.28792
+-9.99159,-10.02870
+-10.06018,9.48071
+10.40320,12.15030
+8.05925,-10.59708
+-10.22557,-9.12809
+-9.72018,9.49238
+10.35544,9.22614
+9.73901,-10.32070
+-9.75240,-10.21685
+-11.46440,9.82556
+10.15484,10.44685
+10.93245,-11.08012
+-10.31915,-8.37950
+-11.22673,9.88504
+12.23454,10.83601
+9.41707,-9.46001
+-10.86017,-9.82758
+-10.53858,11.85271
+10.55666,9.99118
+9.03238,-10.17609
+-11.43390,-9.01418
+-11.34753,8.42938
+10.56948,8.93807
+11.74623,-10.77573
+-11.39278,-10.20016
+-11.42032,8.75999
+11.76026,10.22170
+9.58661,-9.87424
+-9.48506,-9.85621
+-9.40464,10.21250
+9.08462,11.83454
+8.10395,-8.71054
+-10.70364,-10.22778
+-9.62192,9.67795
+8.42368,10.32283
+10.25229,-8.56034
+-9.07037,-9.69116
+-10.79643,9.11136
+10.69799,10.89860
+9.66117,-8.77346
+-9.50474,-9.50682
+-8.66384,10.66286
+9.97451,10.23695
+8.34411,-9.76695
+-8.93502,-10.32524
+-8.80987,9.37580
+10.60491,8.19168
+11.49932,-9.99571
+-9.98837,-10.59546
+-10.05484,10.38143
+10.17492,8.12283
+10.83493,-9.32258
+-9.91615,-9.09390
+-10.34560,9.41936
+10.19901,11.72994
+10.93395,-8.80640
+-8.88071,-8.45264
+-10.26845,8.77118
+10.99892,11.34320
+10.60498,-9.14872
+-11.51901,-10.46141
+-10.64208,10.82576
+9.21683,10.36783
+8.80153,-10.05313
+-11.61625,-11.06702
+-10.58039,9.13377
+9.61062,9.77814
+9.89107,-10.12241
+-10.88774,-10.59721
+-11.62640,10.33853
+10.13162,9.25381
+10.99865,-13.09232
+-10.25090,-9.59662
+-11.84981,10.16305
+8.87558,9.90816
+10.40209,-10.10131
+-9.01751,-9.40874
+-10.62719,9.15865
+10.58668,11.26509
+10.57758,-8.82310
+-10.40943,-9.62288
+-9.61994,9.52770
+10.19491,11.03021
+12.22768,-9.98635
+-8.74716,-9.95879
+-8.72643,9.92323
+10.52184,7.90403
+9.44236,-10.29970
+-10.24023,-11.40987
+-10.85776,12.03655
+11.30847,9.54171
+11.52121,-12.30466
+-11.45129,-8.37280
+-9.92932,9.66063
+9.28605,9.81451
+8.43085,-9.94359
+-10.87157,-10.31406
+-10.80704,11.51600
+10.66732,9.18265
+11.01842,-9.50381
+-9.65104,-10.54543
+-8.39151,10.40849
+8.85969,9.10195
+8.08328,-10.15517
+-9.69741,-9.70019
+-10.64951,11.49262
+9.30059,10.43411
+10.32377,-9.45236
+-9.47512,-10.53571
+-10.95632,8.75729
+9.27227,10.48897
+10.64826,-10.45619
+-10.69654,-9.27361
+-12.02019,9.70297
+10.56431,10.01470
+9.33353,-9.80227
+-7.56565,-11.05551
+-8.77802,11.52510
+10.65183,9.97600
+9.78667,-7.81953
+-8.74087,-11.22424
+-8.89251,9.25861
+9.24100,10.69189
+9.07649,-10.67536
+-9.44931,-9.95896
+-8.89517,11.16744
+10.67267,10.32078
+9.15265,-11.30444
+-10.44261,-10.04051
+-9.15152,11.91603
+9.83571,10.24607
+11.15868,-11.32771
+-11.97593,-10.06097
+-9.75599,11.20916
+11.62991,9.28512
+8.57032,-7.86679
+-10.60814,-10.43671
+-10.92068,8.55943
+8.43994,9.99264
+9.35762,-9.92884
+-10.68265,-7.82498
+-11.36220,9.77553
+8.31742,9.82347
+9.34213,-11.25637
+-10.19078,-8.81159
+-9.50537,9.49779
+10.47606,9.91282
+9.97458,-10.42636
+-11.84820,-10.20320
+-10.29169,7.03393
+9.75931,8.83809
+10.00674,-10.25403
+-10.04610,-7.58333
+-9.69980,9.56736
+8.64410,10.16105
+8.49669,-9.55656
+-10.79407,-9.83092
+-10.25174,10.41187
+11.70579,9.83896
+9.89113,-9.54225
+-9.88159,-10.06377
+-9.23922,10.40577
+10.52956,9.89203
+10.30839,-8.84600
+-9.94331,-10.69225
+-9.73346,8.60950
+9.76000,9.32412
+9.89979,-11.97947
+-8.29955,-8.96930
+-11.11066,8.20145
+10.57363,10.36303
+11.90606,-9.26397
+-9.90191,-11.22731
+-9.36331,9.62364
+9.87795,10.07850
+10.39799,-8.97766
+-9.78584,-11.24788
+-9.79508,10.07134
+8.14681,8.70247
+9.42936,-9.32660
+-9.09625,-10.79427
+-9.55414,11.16000
+9.59010,10.75712
+9.62148,-9.29573
+-11.71576,-11.36847
+-9.49452,6.80102
+9.51125,10.46536
+11.62571,-10.89521
+-11.45188,-8.80009
+-8.36551,10.53777
+10.85440,10.03642
+10.99722,-9.36094
+-7.92822,-9.27086
+-10.79063,11.28557
+11.72977,10.06486
+10.28254,-10.88679
+-9.26859,-8.22547
+-11.14785,10.77090
+10.39917,9.37107
+8.29404,-9.86539
+-10.07251,-9.31353
+-11.13833,8.69814
+10.48512,9.38365
+10.25583,-11.06535
+-10.70262,-9.83535
+-12.07562,9.01573
+9.63468,11.24813
+10.25585,-12.00278
+-11.32571,-9.86184
+-8.02433,10.83352
+10.06404,11.26900
+10.95077,-9.70385
+-8.87787,-10.69507
+-8.93692,10.44702
+11.46571,9.78421
+9.52679,-9.83703
+-9.96772,-9.94943
+-11.80276,9.96528
+10.68934,8.26508
+10.32654,-10.71914
+-9.00847,-9.78940
+-9.95676,9.55562
+10.74881,9.62144
+9.04701,-11.25647
+-11.06152,-11.03243
+-10.52144,10.65656
+10.54649,10.02706
+8.93293,-10.20959
+-9.75555,-8.24950
+-8.88394,9.88851
+11.29424,10.55555
+10.32013,-9.88438
+-8.86706,-10.46414
+-9.86978,9.16246
+9.38677,8.87012
+10.48522,-10.71324
+-9.73145,-10.61888
+-9.30365,10.26243
+8.13151,10.42999
+10.11706,-10.24160
+-11.89146,-10.89536
+-10.75959,9.63612
+9.24869,9.39375
+10.07746,-8.64576
+-10.70202,-9.93297
+-11.47948,9.11574
+9.26885,10.43679
+10.44014,-8.53024
+-12.88640,-10.27430
+-8.90765,10.67841
+9.71475,8.22480
+11.16108,-10.30605
+-9.72922,-10.16904
+-8.90994,8.11586
+11.58402,9.99496
+9.39920,-10.10244
+-10.13973,-10.29796
+-8.87353,10.52496
+11.11657,9.01969
+10.75967,-9.55382
+-11.31207,-10.70274
+-10.22858,11.96016
+9.44268,11.31997
+9.15332,-8.66860
+-9.50601,-10.48623
+-9.80202,8.53985
+10.64897,11.15434
+9.55048,-8.37447
+-11.05853,-9.25759
+-9.42985,9.63768
+9.15252,10.12481
+9.44288,-11.94394
+-8.58459,-10.95423
+-8.28260,10.22162
+9.65815,11.20745
+9.59348,-10.46651
+-10.33206,-10.52959
+-11.18772,9.83934
+10.06477,9.72964
+8.28342,-10.72477
+-10.46652,-12.49823
+-9.94135,9.49195
+9.51470,10.27360
+8.23808,-11.35592
+-10.16097,-9.15651
+-11.78793,9.55070
+9.41413,11.02460
+12.14993,-8.82087
+-8.31218,-9.29350
+-10.35934,11.44261
+8.87646,7.69456
+8.70806,-10.69580
+-11.69875,-10.11620
+-8.60598,11.55969
+10.09524,9.20647
+10.55130,-9.63043
+-10.50279,-9.63517
+-9.18121,9.90207
+9.44163,9.03544
+8.49989,-10.27751
+-9.89247,-10.74012
+-10.18058,10.12641
+10.84224,9.83031
+10.56177,-10.78062
+-9.75046,-10.62538
+-9.65570,10.19726
+9.04907,10.95191
+9.53084,-10.78734
+-10.83672,-8.54925
+-10.07033,10.41792
+9.51839,9.62711
+10.49338,-12.48008
+-11.26310,-10.04565
+-8.97880,10.43018
+10.52719,10.22370
+9.91670,-11.44143
+-11.03951,-9.84144
+-8.53657,9.97058
+10.29459,10.67469
+11.05361,-7.70000
+-11.63766,-10.39472
+-9.89833,10.60257
+9.86405,10.88818
+8.44300,-10.47946
+-8.47597,-8.28325
+-8.54958,10.54535
+10.36523,10.67598
+10.21425,-10.80462
+-9.07898,-10.64414
+-10.10679,10.72486
+11.09691,10.09825
+10.57234,-9.41485
+-9.66385,-10.52620
+-10.39749,10.44148
+10.84375,10.63007
+8.94132,-9.96457
+-10.02531,-9.86802
+-10.47130,9.20389
+11.91318,10.18258
+8.74729,-8.60859
+-11.40144,-11.77743
+-8.70953,10.52690
+9.04428,9.09382
+8.50355,-9.77814
+-10.84473,-10.67358
+-9.61745,9.78373
+9.73847,10.35011
+9.64018,-11.91805
+-11.20755,-10.93271
+-10.66661,8.47735
+10.90053,9.34136
+9.51419,-9.25208
+-10.58943,-10.00315
+-11.24092,9.37299
+10.65184,9.28687
+9.32919,-9.35124
+-8.27000,-8.29687
+-10.61710,10.42892
+11.51309,10.30534
+9.41615,-11.21969
+-9.87554,-8.76790
+-9.68087,10.53741
+11.46531,10.26271
+8.75219,-11.72452
+-10.46305,-11.54305
+-10.61326,9.79287
+8.79629,9.28332
+11.70643,-11.53665
+-8.62419,-9.43471
+-12.59004,11.14481
+11.97597,10.81088
+9.97582,-8.34707
+-8.48314,-11.01478
+-9.12863,10.12450
+10.72364,9.00270
+9.26759,-8.61204
+-9.23810,-10.62061
+-10.11627,10.80092
+10.50190,10.34352
+9.87993,-9.66242
+-9.76975,-11.95072
+-9.30882,10.21236
+9.25384,10.54435
+10.64309,-9.97409
+-9.72175,-10.75629
+-10.22539,10.49177
+9.86606,11.14225
+9.30245,-12.20095
+-11.94751,-9.34538
+-9.85535,10.38128
+9.87582,11.10008
+10.11174,-7.29122
+-11.15510,-10.14839
+-10.52985,10.65736
+10.82806,9.67513
+8.69095,-11.60718
+-8.94200,-10.44717
+-10.79502,10.25254
+9.49914,9.46748
+12.19089,-10.52499
+-9.45773,-9.26559
+-11.67987,9.82782
+10.52943,10.07769
+9.59156,-9.41713
+-8.99695,-9.46368
+-9.89754,8.40402
+9.19110,9.58043
+11.85918,-8.98468
+-10.43832,-10.57873
+-12.20856,10.40840
+11.07069,9.29711
+11.59675,-10.00189
+-9.43323,-11.10929
+-8.95260,10.92276
+10.40581,8.98603
+10.49863,-9.12912
+-9.47945,-10.37257
+-11.02918,10.02553
+10.73109,8.97494
+10.85049,-10.33230
+-11.03759,-10.16282
+-9.88611,12.21666
+11.65297,9.86252
+7.63397,-10.54274
+-8.92631,-10.15829
+-9.64505,9.48019
+9.22521,8.87339
+11.33443,-9.78886
+-10.75998,-10.24247
+-11.33256,10.67410
+9.03171,9.27075
+9.13145,-9.12904
+-10.46049,-8.28460
+-9.92638,10.93481
+9.13505,10.15154
+9.79833,-11.87282
+-11.11225,-10.82600
+-8.65098,11.52105
+9.19391,9.31754
+9.11344,-8.84661
+-10.25726,-9.66359
+-9.44136,10.11502
+9.42663,9.04228
+10.96849,-7.98245
+-11.08900,-9.33322
+-8.86629,10.28241
+8.75009,9.41275
+10.25725,-10.31360
+-9.89851,-9.86326
+-10.20335,10.20924
+9.19968,9.37012
+8.46887,-8.69715
+-9.79705,-10.58431
+-9.05459,10.54880
+9.81111,9.21532
+10.00949,-11.06324
+-9.71772,-10.03718
+-10.39272,11.30592
+12.26283,12.47808
+9.96376,-9.82196
+-11.14256,-8.61028
+-11.55581,9.08318
+8.70474,10.33340
+11.00542,-10.16020
+-11.11621,-10.38741
+-10.99130,9.72802
+9.91788,9.70616
+9.60870,-11.81455
+-10.11210,-11.97700
+-10.47717,8.59196
+9.78242,9.74894
+10.82395,-11.18915
+-9.69018,-10.66239
+-10.01874,10.10292
+11.12349,9.72867
+9.64404,-9.35266
+-9.99538,-8.34715
+-9.10127,10.00495
+8.66800,10.15992
+11.29981,-9.31946
+-11.69894,-9.51943
+-9.21862,10.38483
+7.85643,11.64132
+9.62455,-9.61575
+-10.80494,-9.17660
+-9.16776,9.12813
+8.93215,8.36194
+9.44088,-9.67764
+-10.55901,-9.99324
+-9.38533,9.40505
+11.15936,10.62704
+8.07778,-11.22977
+-9.78534,-9.31984
+-9.37392,9.04891
+9.57906,10.49602
+9.58480,-10.37474
+-8.39454,-10.45993
+-8.89268,10.15450
+8.47869,9.64489
+9.55856,-10.42331
+-11.42012,-9.62275
+-7.91515,11.44922
+8.06418,10.17037
+9.65402,-9.35310
+-7.59222,-11.27554
+-8.86482,7.13143
+9.65807,8.23778
+11.83169,-11.67349
+-10.24409,-9.05949
+-9.09672,9.82468
+8.96103,11.61722
+10.40288,-10.80640
+-9.22883,-10.49838
+-9.54583,11.93759
+8.96811,9.61960
+12.30465,-10.79016
+-11.20479,-8.83767
+-10.45555,10.34613
+9.58144,7.47014
+8.96198,-9.64458
+-10.48451,-10.75737
+-9.92446,9.32209
+9.08451,9.84475
+9.47230,-8.76557
+-9.35564,-11.19151
+-11.19418,10.17918
+10.14386,10.48677
+8.19325,-9.82568
+-10.35575,-10.87041
+-10.02971,9.91164
+9.00833,7.90113
+11.31398,-10.57070
+-9.30382,-9.87118
+-8.25801,9.42021
+10.34579,9.56073
+8.44147,-9.19147
+-9.27076,-10.89507
+-10.59710,9.54343
+9.88785,9.33464
+10.63661,-10.16920
+-12.54280,-9.39922
+-9.14928,9.77207
+10.80544,10.09376
+9.96543,-9.33683
+-10.31618,-9.42131
+-9.74002,9.88686
+9.45474,10.44028
+9.22900,-9.89494
+-10.47496,-10.84587
+-9.09548,8.53419
+8.76685,10.51129
+10.67755,-10.59017
+-8.86704,-9.43568
+-10.91029,10.03179
+11.96466,8.35327
+11.32226,-7.54650
+-9.55566,-9.58530
+-9.73523,8.91382
+9.31501,10.93116
+10.70222,-9.45918
+-8.67188,-11.07144
+-9.63043,10.49509
+10.29285,7.13558
+9.16630,-11.55163
+-9.51891,-9.27304
+-10.42674,10.22133
+7.65101,10.49334
+9.92932,-9.94036
+-8.84212,-10.30083
+-10.14162,10.53561
+10.27343,9.57253
+7.61137,-8.52485
+-10.69527,-9.28493
+-8.78927,9.90368
+11.22695,10.54092
+12.21694,-8.36452
+-9.43872,-11.03264
+-9.89630,10.93669
+9.50124,10.46077
+8.85707,-10.86353
+-9.39820,-8.78256
+-8.58243,11.32020
+9.29245,10.48441
+10.68084,-9.96768
+-11.32185,-8.84007
+-8.17034,10.31722
+9.97534,10.63635
+9.59076,-8.79146
+-9.27870,-10.67988
+-9.27685,10.10192
+9.64749,10.12262
+8.77071,-10.46885
+-9.07925,-9.37960
+-12.10500,9.53437
+8.42484,9.14261
+10.30229,-10.41343
+-10.92112,-10.42160
+-10.65351,10.39823
+10.06882,11.38246
+10.10279,-9.06921
+-9.67704,-11.08654
+-10.08718,10.99317
+10.30436,10.26325
+8.76435,-8.60560
+-10.27546,-10.31912
+-10.02422,10.26034
+9.55263,9.41495
+10.63273,-10.56018
+-8.55471,-9.19257
+-11.32008,9.83447
+9.74225,10.04438
+9.97191,-8.33492
+-9.95271,-8.98064
+-9.27597,9.17294
+8.88248,8.85009
+9.78840,-9.56087
+-10.71759,-8.46046
+-9.87034,9.97753
+9.95569,8.72420
+10.08893,-11.46410
+-8.49964,-10.30004
+-9.03975,9.09678
+8.79803,8.38859
+11.07869,-10.85340
+-11.10759,-9.56920
+-12.00037,11.32083
+10.24790,10.44916
+10.39938,-9.96021
+-11.61014,-8.51105
+-10.06365,10.59015
+11.06871,11.08913
+8.61891,-7.55900
+-9.32947,-9.27999
+-9.11732,9.58000
+10.48918,8.97188
+9.95535,-9.23402
+-8.88180,-12.79965
+-10.47560,10.79120
+12.80205,10.95027
+9.41133,-9.71040
+-9.77903,-9.03413
+-10.31487,8.93143
+7.70913,10.78127
+10.28634,-8.21118
+-10.69783,-9.49457
+-10.26240,9.51839
+9.84185,11.69327
+11.44220,-10.10115
+-10.30142,-8.74080
+-9.90442,10.05154
+11.11607,9.99819
+7.60004,-10.80361
+-8.26430,-11.21276
+-10.92996,9.40710
+11.12296,10.77676
+10.63772,-11.57512
+-10.05900,-10.65579
+-8.44292,10.46103
+9.95266,9.36789
+9.94675,-11.28778
+-9.99341,-10.62477
+-9.86851,11.20962
+9.68328,10.35880
+10.27559,-7.66900
+-10.11344,-10.01329
+-10.99477,8.89559
+10.73109,9.01820
+10.67914,-9.26379
+-10.70825,-9.20894
+-8.99756,10.39753
+11.28089,11.92408
+11.31259,-9.63516
+-11.57868,-10.48557
+-9.66125,9.21492
+10.29112,10.00245
+9.91586,-8.98206
+-11.76764,-10.55544
+-9.47803,9.18555
+8.87493,10.31171
+10.65364,-10.29233
+-9.24622,-11.35248
+-10.17758,9.53337
+12.38367,10.94936
+9.03410,-9.80377
+-10.52415,-10.09690
+-9.08302,8.77263
+9.91066,9.06123
+10.50949,-10.51057
+-10.45665,-8.52442
+-10.92496,10.70251
+9.64081,10.90970
+9.63660,-8.65856
+-9.74226,-7.99795
+-8.98660,10.84586
+10.98563,11.55297
+9.41755,-9.71777
+-9.31225,-8.11843
+-11.15342,10.73089
+11.51908,9.40172
+9.87708,-9.53170
+-6.29777,-11.07460
+-8.43269,10.57460
+10.84711,9.70584
+9.89754,-12.33110
+-10.05182,-9.77429
+-9.70191,8.68897
+10.97432,8.88839
+9.46577,-10.96167
+-10.92212,-9.57492
+-8.91674,10.04743
+9.25385,11.20176
+9.80087,-10.97399
+-9.37582,-10.20886
+-10.28980,11.75816
+9.43693,10.98890
+8.08177,-10.82643
+-10.27455,-8.86069
+-9.18429,10.07038
+12.40674,9.03186
+8.17993,-9.93426
+-11.13369,-10.35025
+-8.67991,9.32139
+9.38490,10.35212
+12.37134,-8.93799
+-10.02126,-9.73579
+-10.21148,10.94790
+10.21855,11.12301
+11.84520,-9.95379
+-10.08219,-9.85965
+-10.86422,9.02657
+8.80441,8.88312
+9.38765,-8.35416
+-10.58772,-9.96677
+-9.83424,9.36373
+9.89480,9.60032
+9.91690,-10.45572
+-10.95620,-11.47335
+-10.46923,10.19339
+9.81536,10.54988
+9.75902,-9.88850
+-11.02320,-10.60687
+-11.59303,11.25902
+10.31803,9.80335
+9.26843,-9.46147
+-10.47596,-7.94780
+-10.91056,9.13315
+10.40092,9.08614
+10.16462,-10.73271
+-11.00682,-10.12169
+-8.25845,11.09830
+11.15132,8.85841
+9.44961,-11.34660
+-10.60552,-10.05685
+-9.43141,11.20916
+11.03772,8.59279
+12.02925,-13.05519
+-10.11750,-10.91216
+-9.85988,9.35512
+9.91418,6.83700
+11.28332,-10.48652
+-8.81768,-9.68327
+-9.81869,11.13233
+10.06736,9.66354
+9.34674,-8.82519
+-9.82858,-9.74127
+-8.58446,11.46129
+9.09268,10.82585
+7.71658,-8.47863
+-10.22813,-10.70833
+-9.61491,10.49594
+10.62511,10.31107
+9.15085,-8.81277
+-10.97890,-10.26417
+-9.59423,8.22571
+10.14700,10.32696
+8.15235,-10.42143
+-9.43293,-11.25834
+-10.95003,10.91845
+9.17908,10.52759
+9.11283,-9.41535
+-11.00299,-8.46671
+-10.06652,11.49729
+10.32334,9.98745
+9.41831,-11.12405
+-9.81627,-9.74843
+-8.03700,10.16260
+10.61799,9.83387
+9.87311,-10.87389
+-10.75885,-9.08786
+-11.04535,9.55710
+9.46723,8.37607
+10.44379,-8.37277
+-10.61167,-9.28459
+-10.02661,10.18759
+9.98924,9.22409
+10.06916,-10.08374
+-9.48052,-9.03765
+-9.62782,9.51170
+10.39128,8.38659
+9.54395,-9.21258
+-9.65792,-8.13955
+-10.38723,9.00748
+10.01453,11.26385
+10.51083,-10.87067
+-8.06241,-11.80947
+-10.21736,9.35373
+9.58856,9.80139
+9.77929,-10.34641
+-9.09253,-8.62115
+-11.02266,10.59942
+9.71770,9.65174
+9.77391,-9.43664
+-10.45995,-9.46080
+-10.45731,11.86855
+11.32300,9.83960
+11.15035,-9.73065
+-9.74240,-9.56888
+-8.36787,10.33475
+9.25261,10.51227
+8.74993,-10.04950
+-10.13450,-9.60391
+-10.67909,10.14401
+10.73997,11.59405
+9.82971,-9.07748
+-9.82335,-9.22573
+-9.19710,9.16825
+9.24665,10.43608
+10.88274,-10.56076
+-10.03898,-9.55706
+-10.47067,9.94035
+10.43429,10.23103
+10.35409,-10.60652
+-10.25876,-10.30415
+-9.72199,8.85331
+11.15065,8.77434
+9.47208,-10.40264
+-8.53253,-9.07565
+-10.33690,11.49682
+11.67307,9.91078
+9.73898,-9.82254
+-10.42165,-10.88186
+-10.83585,10.00588
+8.78586,10.84840
+9.90845,-8.99338
+-10.79441,-10.47504
+-9.16114,9.16882
+9.55570,9.44912
+10.08366,-10.22570
+-10.85012,-11.68615
+-10.68070,8.64106
+9.50404,10.61205
+9.40082,-10.13610
+-9.89009,-11.63597
+-10.74468,12.21174
+11.50419,9.06713
+9.15049,-10.29446
+-7.10964,-8.88147
+-9.82546,11.14412
+10.62281,9.48242
+9.48685,-10.84159
+-7.46787,-8.23574
+-10.31850,8.86347
+9.17727,10.88576
+9.60150,-9.26071
+-9.23287,-9.22241
+-8.62883,12.41553
+9.32057,11.80365
+10.64323,-9.77647
+-9.69339,-8.93396
+-9.27680,10.54382
+11.28816,9.33641
+10.28673,-9.20076
+-9.92438,-9.97501
+-11.49619,10.42339
+8.73991,9.54494
+9.70693,-9.04184
+-11.39584,-11.34730
+-12.39745,10.31759
+10.06126,10.12818
+11.03006,-9.40169
+-10.88447,-8.57841
+-11.03673,9.79407
+10.08636,10.65665
+9.64523,-9.91586
+-9.29607,-10.84050
+-11.07979,9.73751
+10.26268,11.06429
+10.51854,-9.92436
+-12.08195,-10.43921
+-9.91459,8.79562
+10.45551,10.26713
+9.68833,-11.00648
+-9.52510,-9.84927
+-9.94212,11.58458
+11.28707,12.08910
+10.31708,-12.30610
+-9.73976,-10.08438
+-12.08798,11.03870
+8.97444,10.58669
+8.96024,-9.31396
+-9.86528,-10.66677
+-10.47759,9.47765
+9.60334,9.82741
+8.99051,-10.07203
+-8.12237,-9.17812
+-10.08703,9.97924
+8.76335,10.34064
+9.76502,-9.80477
+-10.43414,-10.13372
+-10.71287,10.25450
+10.47397,10.61440
+8.86843,-9.43281
+-9.32755,-8.66192
+-9.45440,9.21143
+10.40434,10.51786
+9.88957,-10.54055
+-8.95857,-9.99659
+-10.71157,9.61161
+11.37256,12.31619
+8.31390,-9.99586
+-10.97374,-11.02039
+-11.72044,10.17927
+10.46939,9.64374
+9.70007,-9.32127
+-9.75897,-10.03636
+-10.36969,9.51455
+9.33637,9.99405
+10.52599,-10.79201
+-9.57387,-10.48218
+-11.01860,10.59234
+8.45823,9.84734
+10.03451,-11.38201
+-8.58621,-9.31860
+-11.33038,8.79039
+10.93750,8.33136
+9.61437,-10.56940
+-10.71779,-11.18660
+-10.68798,10.55241
+12.63032,11.21240
+9.74593,-10.32292
+-10.68585,-10.00718
+-9.41725,9.37669
+9.61719,10.08252
+9.12815,-9.27503
+-9.03102,-9.19192
+-9.75716,8.13877
+8.41635,11.23297
+10.15648,-8.64558
+-9.24258,-11.39367
+-6.55087,10.01537
+10.85974,9.35423
+11.31797,-10.80303
+-13.14369,-8.75669
+-9.57113,10.53832
+8.76127,8.96049
+9.73291,-8.19976
+-11.42211,-8.31483
+-10.05291,8.90115
+10.80236,9.67388
+10.41112,-11.33740
+-9.82447,-11.34505
+-9.50285,11.59104
+7.96122,10.56261
+9.00493,-11.04205
+-9.79534,-9.60143
+-8.74642,10.36183
+11.67578,9.84070
+8.44504,-9.87919
+-10.60747,-9.81603
+-9.69835,9.28428
+8.78769,10.09776
+8.44880,-8.91534
+-10.46936,-10.13421
+-10.50303,10.74009
+9.97225,11.59172
+8.42933,-10.80522
+-9.16256,-11.57247
+-8.93642,8.86322
+10.68341,8.91575
+8.19499,-10.83781
+-10.31156,-12.72497
+-10.19321,11.00126
+11.37743,9.09325
+10.94845,-9.54230
+-10.40616,-11.15211
+-10.60855,10.88800
+11.94946,9.99348
+11.48136,-10.44714
+-10.78956,-12.19422
+-10.52365,10.80978
+7.61346,10.93574
+10.30666,-10.85650
+-8.59686,-9.52403
+-11.46745,12.17922
+7.56966,9.95405
+8.83685,-9.85774
+-12.09157,-9.26543
+-9.40324,10.10748
+11.74807,8.25345
+8.27080,-8.43264
+-10.35350,-9.60915
+-12.22876,10.40878
+9.87733,8.94611
+10.74486,-9.50146
+-9.52517,-9.69582
+-7.82399,10.67091
+8.53566,10.44915
+8.56021,-10.90427
+-10.03729,-11.08425
+-10.11000,9.99658
+10.86478,8.41811
+10.92702,-8.31874
+-10.64274,-9.05414
+-10.42544,12.55736
+10.58851,9.09031
+10.74173,-10.69060
+-9.68754,-11.54336
+-9.62220,10.89999
+9.21102,10.37279
+12.69146,-9.97716
+-10.31971,-10.31722
+-8.49202,8.65146
+9.48482,9.88730
+9.49365,-9.01203
+-10.83574,-10.74087
+-8.45679,10.45765
+10.11812,8.14800
+10.18153,-10.13842
+-10.88526,-8.56737
+-9.65828,11.33459
+8.96792,9.26879
+9.35153,-10.39398
+-11.03505,-8.78654
+-9.61227,9.45649
+10.29319,10.51988
+9.23858,-9.65536
+-8.56124,-10.81826
+-10.16739,9.24695
+10.07865,8.73473
+10.63016,-9.64548
+-10.91359,-9.48003
+-9.52119,8.94110
+10.81665,10.21175
+10.03912,-9.95308
+-8.86059,-10.07283
+-9.80386,10.17100
+8.96885,9.59567
+11.23714,-9.89384
+-10.84836,-12.02421
+-10.70033,11.40807
+9.85420,8.72737
+9.59864,-9.64821
+-10.11140,-8.39945
+-8.48537,8.70599
+7.17319,10.25497
+8.43546,-10.70105
+-9.49379,-10.61272
+-8.32888,7.95118
+10.48001,8.96229
+10.09136,-12.01396
+-9.35265,-9.21129
+-9.96562,10.70058
+9.93070,10.43543
+9.80635,-9.42739
+-10.30613,-11.41472
+-9.54824,9.57979
+10.66840,8.43768
+10.60413,-9.64198
+-10.48062,-11.96045
+-11.28785,8.68804
+10.66790,10.49686
+9.61687,-9.91093
+-9.40255,-8.13703
+-10.14811,12.15044
+10.44432,9.66270
+9.12352,-11.68014
+-10.84510,-9.58625
+-10.69162,9.02011
+11.08409,9.35912
+9.44219,-8.84953
+-10.87934,-10.09438
+-10.54872,9.57054
+10.77447,10.64440
+10.06530,-11.27434
+-8.95712,-10.62679
+-10.11518,9.26384
+8.70536,10.73668
+9.40769,-8.31787
+-8.74686,-9.25260
+-10.29313,9.74620
+10.80250,8.90296
+11.49037,-11.09867
+-10.06816,-11.58865
+-9.16050,10.46227
+10.25959,9.98356
+9.39850,-12.65729
+-11.61478,-8.83689
+-12.10768,10.63044
+11.74971,9.22958
+10.81782,-9.44097
+-8.96008,-11.12738
+-9.56673,11.33469
+11.07727,9.33975
+10.70519,-9.33468
+-10.90292,-10.94321
+-10.20389,8.66395
+7.61353,10.17079
+8.16764,-10.12468
+-10.76798,-9.88147
+-8.22346,10.09289
+8.14600,9.11745
+8.97772,-9.70741
+-9.60056,-9.65860
+-10.84268,10.06467
+9.02425,10.65266
+9.00492,-11.00719
+-10.21121,-9.31989
+-8.55302,8.04990
+9.97321,12.52390
+9.46405,-8.35804
+-10.64161,-10.21900
+-11.19039,8.97472
+9.93847,10.09176
+9.60094,-10.52043
+-10.27676,-9.20620
+-8.36384,9.37933
+8.91720,11.68669
+9.16616,-11.33913
+-9.39898,-9.28998
+-11.23349,10.69344
+9.74775,9.27126
+9.69480,-8.70615
+-10.36079,-10.18637
+-9.10165,8.03978
+10.04962,10.13120
+11.41063,-9.37674
+-9.45208,-10.90411
+-10.17537,10.52426
+11.32753,9.57711
+9.72533,-8.90241
+-8.73365,-10.27658
+-9.03294,11.22799
+10.94752,10.78691
+8.87296,-8.99359
+-9.47792,-9.10432
+-10.98946,10.23363
+10.92955,11.19224
+9.05473,-10.73379
+-9.29672,-10.42600
+-9.24922,9.95435
+10.62167,9.35227
+10.40374,-11.42599
+-9.59486,-9.30061
+-8.73411,10.16797
+10.68065,9.50194
+9.12940,-10.08002
+-13.20266,-10.50139
+-9.28253,10.25246
+11.76255,9.07893
+10.54231,-9.50831
+-10.75463,-10.93539
+-10.05028,8.79016
+8.85172,9.86192
+10.73192,-11.94862
+-9.31136,-11.33658
+-10.11581,10.48971
+8.30630,8.98931
+9.82517,-9.50889
+-10.96240,-9.07125
+-8.34161,10.17435
+11.01421,10.58141
+11.50779,-10.57180
+-9.84470,-10.77894
+-10.43503,13.50708
+10.70675,8.55659
+9.38325,-10.16095
+-9.20017,-10.52166
+-9.97910,10.52918
+10.68008,8.77263
+10.78693,-9.99600
+-10.54018,-10.72031
+-11.14826,10.64416
+7.01139,10.48929
+9.76177,-9.61969
+-10.37139,-10.55938
+-9.76813,10.93745
+9.73541,9.33614
+9.29969,-8.60385
+-11.22900,-9.20412
+-9.86335,8.99907
+10.81002,9.59209
+9.61957,-8.79551
+-9.78052,-10.36891
+-9.83401,8.79355
+11.84055,8.97200
+10.51734,-12.15276
+-9.76229,-11.09928
+-9.91316,10.17002
+9.75169,10.35659
+11.61924,-9.57015
+-10.42036,-10.92602
+-9.35982,8.64258
+10.77571,10.18077
+10.64454,-11.19855
+-9.91067,-9.72856
+-10.34776,10.13313
+10.07569,9.73453
+10.58006,-10.40248
+-9.68769,-11.29593
+-9.02443,10.27002
+9.18493,10.34293
+10.35502,-11.16033
+-11.03656,-9.78116
+-10.08645,10.70801
+10.58221,9.90256
+11.06925,-9.31761
+-11.10598,-8.72446
+-10.83196,10.44623
+11.23501,10.71827
+10.94620,-9.49507
+-9.48766,-10.09482
+-10.39940,11.32019
+10.76782,8.66913
+10.24857,-9.89298
+-11.96983,-9.34175
+-10.14833,9.24922
+10.00450,9.12574
+9.62813,-11.48850
+-11.39994,-9.92385
+-11.95881,9.82997
+8.74121,11.85682
+9.02804,-11.18860
+-9.94818,-10.02869
+-9.63564,10.67136
+10.47073,10.04878
+10.65266,-9.18219
+-10.00678,-11.04424
+-10.80863,9.48840
+10.39855,10.33551
+10.73063,-9.91910
+-10.35383,-9.77265
+-10.85487,9.79282
+9.05376,11.13943
+10.68964,-9.33286
+-9.78195,-10.89560
+-10.63017,8.46919
+10.76136,9.53922
+9.67453,-9.24154
+-9.96724,-10.46162
+-8.94527,8.84043
+8.31810,9.93529
+10.03794,-10.57478
+-9.88380,-9.80227
+-10.83276,10.45736
+11.61800,9.78253
+9.00654,-8.98196
+-11.97317,-11.21872
+-8.04118,10.17952
+10.63615,9.93945
+10.64971,-10.28937
+-9.77323,-8.83973
+-11.56613,9.98205
+10.51620,11.01738
+10.56974,-11.19665
+-8.39982,-8.41764
+-9.49641,11.75914
+10.64531,10.56239
+11.41540,-9.54791
+-9.34147,-11.03608
+-7.56534,9.25112
+9.56555,10.04900
+11.00549,-11.86503
+-10.58790,-9.95768
+-10.23226,9.24464
+9.67237,8.09846
+10.35730,-11.36775
+-10.21473,-9.87388
+-9.62621,9.71467
+6.94730,9.59390
+11.03794,-9.33001
+-11.01380,-9.71079
+-11.09968,11.99798
+11.34021,10.16110
+11.06726,-11.33500
+-9.83756,-8.93091
+-11.93357,10.32369
+9.64440,10.10208
+11.56364,-9.47694
+-8.95051,-10.11549
+-10.83714,8.97137
+9.24766,9.10578
+10.42665,-10.22745
+-8.18221,-9.41196
+-10.79040,9.70555
+8.27612,9.80045
+10.88386,-9.31879
+-8.18882,-11.48907
+-8.88558,9.16517
+11.58991,9.97964
+10.56549,-9.70217
+-11.48761,-10.93482
+-10.18730,9.77166
+8.54283,8.09089
+9.51150,-11.36022
+-8.43463,-9.68420
+-11.17303,9.15176
+8.39843,9.76912
+9.05198,-9.40900
+-11.08811,-9.99038
+-10.25020,11.16444
+11.57667,9.27660
+8.51229,-8.23926
+-10.00824,-9.71053
+-10.56341,10.51602
+11.09616,9.81538
+9.67911,-8.64728
+-9.45880,-9.65452
+-11.72345,8.35724
+12.04844,9.74746
+9.98079,-10.87341
+-10.36152,-8.19307
+-8.33232,10.95389
+11.09366,8.58392
+9.24920,-9.64713
+-10.39888,-11.92003
+-10.01684,9.20921
+9.11519,11.62309
+9.83804,-11.19937
+-10.07707,-10.67021
+-10.15440,10.57642
+8.86789,10.21159
+11.04362,-7.28457
+-9.60134,-10.55117
+-10.43157,9.38632
+10.54607,10.45835
+10.33942,-10.70865
+-9.54187,-9.14477
+-9.58951,11.12467
+10.55202,10.04780
+9.14376,-11.88945
+-10.75022,-8.71338
+-11.33146,10.52476
+9.27969,9.24963
+9.06059,-9.81043
+-10.54425,-11.85050
+-9.25384,9.07800
+9.90000,9.65390
+9.82990,-11.19136
+-9.97897,-8.95809
+-10.91102,9.41243
+10.45262,9.28627
+12.16781,-9.86538
+-10.28557,-11.09891
+-10.98093,10.00284
+9.51447,9.70696
+10.02521,-10.35146
+-10.64041,-9.53112
+-9.73720,9.90968
+11.15725,10.32555
+10.28386,-10.01604
+-10.70376,-9.10313
+-7.48772,10.90613
+9.44219,11.64441
+7.91728,-10.87334
+-9.96229,-9.16947
+-10.35288,8.44694
+11.09151,9.90016
+9.04765,-9.37088
+-10.79723,-10.63626
+-8.27109,10.17326
+9.56313,10.92483
+10.46130,-9.56259
+-11.57197,-8.97484
+-10.94328,11.22025
+8.88416,9.62187
+11.69446,-9.55697
+-10.90965,-9.52572
+-11.08771,11.66433
+11.56181,10.74386
+9.61245,-10.28202
+-11.68226,-10.24917
+-11.77268,11.39983
+10.51782,9.97901
+12.75965,-11.05466
+-9.70766,-11.13685
+-9.42090,9.88181
+9.38191,9.79542
+9.17566,-8.68105
+-9.55238,-10.28186
+-9.44308,9.30177
+9.33371,10.38209
+10.95704,-11.14292
+-8.84284,-10.58917
+-9.53155,10.08764
+10.23505,9.86940
+9.75646,-10.56245
+-8.78463,-9.75651
+-9.86977,11.26280
+8.76890,9.72466
+8.51864,-10.00896
+-10.22209,-11.44049
+-11.73588,10.75983
+9.47205,8.22890
+10.28573,-11.12865
+-9.07751,-10.12536
+-10.77210,9.60531
+9.10633,9.49156
+9.57739,-9.52363
+-9.94784,-10.96712
+-11.86352,11.25487
+10.16170,9.30559
+9.96664,-9.28767
+-7.60822,-10.05662
+-8.48897,9.77462
+11.49891,10.46087
+10.74955,-10.36553
+-10.27367,-9.81266
+-9.08125,10.68571
+9.37561,9.62865
+10.78315,-8.26203
+-10.57117,-8.44187
+-10.52111,9.87612
+11.32233,10.13652
+10.53286,-10.21930
+-8.30090,-10.54045
+-7.90994,11.08974
+12.01670,10.35729
+10.64468,-11.14721
+-10.47633,-9.87038
+-10.47067,10.83801
+11.23762,9.79559
+9.51695,-9.58565
+-9.96302,-8.37489
+-9.30486,10.13033
+10.13418,10.21406
+9.38703,-9.43859
+-10.63405,-10.26515
+-11.52662,10.38265
+9.76865,11.58319
+11.23318,-10.13285
+-12.31598,-9.89691
+-11.62461,8.65826
+11.52337,9.25706
+11.84452,-10.53580
+-9.85142,-9.62015
+-10.00215,10.08419
+9.78511,11.01870
+9.60777,-9.87978
+-9.42086,-9.21947
+-7.20827,9.77623
+11.93197,10.19388
+9.30699,-9.55801
+-11.59384,-9.32208
+-10.70572,11.68779
+10.48862,11.27397
+8.49983,-9.51112
+-10.13931,-9.69284
+-10.07403,8.32104
+9.66350,10.61524
+8.78344,-9.51390
+-10.73077,-10.62041
+-10.60577,9.13588
+12.15041,9.12974
+8.28330,-11.15637
+-10.49889,-10.42010
+-10.41516,8.95766
+9.38933,11.87899
+9.45431,-9.78214
+-9.69391,-9.71368
+-9.02130,11.36898
+10.24299,9.18835
+11.32681,-9.52304
+-10.59800,-9.95048
+-10.49275,10.54310
+9.18342,9.48362
+9.75355,-8.97105
+-10.44823,-11.06990
+-10.98686,9.75176
+10.85274,8.63300
+10.89502,-9.62987
+-11.24516,-9.36583
+-9.03830,9.75536
+9.44605,9.13859
+9.88730,-9.36030
+-8.68209,-9.35414
+-9.78983,10.17685
+9.11946,10.32151
+9.12629,-9.89136
+-10.77053,-12.21757
+-9.27509,9.10914
+7.85301,11.08581
+9.86240,-9.43921
+-10.50512,-12.02590
+-8.71647,10.09409
+12.34203,10.91697
+9.86103,-10.32208
+-8.44384,-9.81000
+-9.88393,10.19324
+9.46963,10.01902
+9.53257,-9.57480
+-9.37446,-10.00383
+-9.41741,8.76450
+10.55942,8.81034
+9.48070,-9.36460
+-10.19090,-10.39638
+-11.17095,9.18750
+11.09428,9.41816
+7.15176,-7.48009
+-10.98064,-10.10430
+-12.29053,8.10914
+11.52617,9.39782
+10.79458,-10.92674
+-10.02957,-9.41767
+-11.30642,10.43524
+9.61500,9.53147
+9.24280,-9.47896
+-10.06233,-9.52917
+-10.56459,10.16972
+11.07011,8.88614
+10.64789,-11.02875
+-11.23482,-9.77374
+-10.21381,9.43061
+9.53265,10.29664
+10.14220,-11.68607
+-8.98046,-8.59143
+-10.65376,10.92609
+9.96468,9.65312
+10.82233,-11.46105
+-10.91044,-8.74640
+-9.54067,11.32654
+10.33617,9.59316
+10.10908,-11.06026
+-9.59061,-8.70149
+-10.84619,11.32652
+11.37198,8.84661
+9.41938,-10.90601
+-11.46386,-9.29073
+-9.97907,8.70585
+11.58640,9.23860
+10.96691,-10.38890
+-10.68422,-10.28405
+-10.88276,9.63828
+9.30272,9.81604
+9.01993,-9.11085
+-11.27351,-9.94921
+-10.55258,8.91758
+9.80545,11.00270
+9.94645,-11.66720
+-10.12939,-8.77013
+-9.36375,9.51837
+9.36611,10.57060
+10.36540,-8.28415
+-10.83033,-9.51142
+-8.44449,7.45911
+9.08560,9.56849
+9.90975,-9.79785
+-7.49064,-9.59194
+-10.44850,10.35319
+10.58404,8.23014
+9.65314,-10.76063
+-10.25318,-10.01815
+-10.99229,12.53927
+10.42115,10.37066
+10.75562,-8.36771
+-10.14640,-9.21342
+-8.77527,8.95526
+10.54213,9.84744
+11.12978,-9.50050
+-9.19697,-10.54157
+-8.42113,8.80230
+9.78875,8.87159
+10.04474,-11.16343
+-9.06645,-9.70890
+-9.48233,11.83288
+9.86798,9.18452
+12.25316,-9.20018
+-7.97615,-9.31311
+-11.05973,11.21298
+10.21156,10.45430
+9.75581,-9.15357
+-11.24647,-10.96285
+-10.05114,9.23251
+12.61478,10.40502
+10.95653,-11.01946
+-11.70804,-9.33502
+-10.29793,11.47609
+10.92895,9.97216
+10.66593,-9.06169
+-10.52939,-10.32693
+-10.49486,10.20875
+8.95912,9.80409
+9.10768,-11.26651
+-8.69612,-11.25141
+-10.45940,8.70423
+11.86130,11.83546
+10.50039,-10.94070
+-10.34670,-8.77811
+-9.34305,10.15918
+10.42983,8.04248
+9.57714,-12.11802
+-10.45606,-8.65999
+-10.04789,11.39770
+7.72869,10.26550
+8.70939,-10.15386
+-9.38694,-10.73249
+-11.21435,9.25535
+10.33313,9.45661
+10.52992,-9.47621
+-9.63197,-10.38560
+-10.58455,9.67180
+11.95621,10.55346
+10.57743,-8.43214
+-10.38449,-9.43133
+-10.68516,9.80200
+9.00333,9.94810
+8.86757,-9.18376
+-9.90721,-8.43154
+-11.49413,9.75009
+11.05685,9.10761
+10.85838,-9.94840
+-9.15183,-11.25168
+-9.14148,10.51472
+11.61636,9.13256
+10.44138,-8.61729
+-10.74991,-10.81455
+-8.72055,8.97516
+9.95070,10.21997
+10.28133,-10.87572
+-9.65229,-9.50776
+-8.35124,11.27332
+9.23484,11.17286
+9.95753,-11.35888
+-10.22205,-8.66579
+-9.18190,10.52779
+8.96562,10.35076
+10.05143,-8.62515
+-8.65531,-10.88067
+-9.91038,10.21971
+9.22634,10.02385
+12.75584,-11.34777
+-11.19335,-9.68508
+-10.59324,12.73677
+9.27144,8.85628
+9.48442,-9.34458
+-9.57836,-8.13542
+-9.66251,10.66762
+9.64036,10.18965
+11.12328,-10.57904
+-10.13614,-10.37851
+-8.68083,10.33765
+9.82053,10.79880
+9.74320,-9.14125
+-10.64042,-10.33603
+-10.21994,8.95652
+9.74667,10.25153
+10.55977,-9.30341
+-8.48899,-13.43006
+-9.64903,11.25250
+9.67518,10.17905
+10.85232,-10.36268
+-9.63307,-9.01910
+-8.97169,9.73714
+9.65828,10.34453
+11.72294,-9.15263
+-9.53102,-8.53525
+-9.10472,10.98810
+9.88719,8.40953
+9.54840,-9.74921
+-10.17453,-11.15416
+-9.14789,7.92542
+9.34377,10.39611
+10.02921,-8.78428
+-11.19726,-9.84175
+-8.36984,11.66586
+9.30787,10.10312
+8.39019,-10.29648
+-9.35407,-9.93836
+-8.23039,10.30656
+9.61505,9.84766
+9.30512,-9.06253
+-8.59435,-9.73055
+-10.31132,9.28898
+9.92473,9.33274
+9.69384,-11.70697
+-11.46422,-9.68404
+-9.13284,9.89811
+10.01849,9.45469
+9.43593,-9.51367
+-9.50948,-10.58038
+-8.81880,11.78332
+9.87561,10.38491
+9.58383,-9.01125
+-9.31607,-8.35443
+-9.45166,10.12270
+10.35946,8.94101
+8.28238,-8.93048
+-10.48910,-9.28016
+-9.22958,9.89736
+8.26727,12.84017
+9.75653,-10.87686
+-10.29979,-11.15305
+-9.74975,9.07300
+10.47902,11.34847
+10.71455,-8.26415
+-10.86554,-10.40597
+-11.78749,11.90399
+9.84439,12.38904
+7.97185,-9.26923
+-11.61841,-10.59871
+-10.64477,10.40737
+10.06693,9.70035
+9.59082,-10.95631
+-9.45486,-10.13228
+-9.17834,11.44926
+11.33755,9.28238
+9.42653,-9.45048
+-9.42614,-9.63492
+-11.42318,10.99099
+10.23419,8.60358
+7.78590,-10.80084
+-8.91957,-10.48927
+-9.18596,9.50772
+8.99929,9.41290
+10.01870,-10.01520
+-11.25058,-9.05332
+-10.27144,10.35523
+9.44682,10.58225
+8.65837,-12.14254
+-11.50614,-10.52733
+-9.07863,11.32192
+10.59914,10.54854
+9.48065,-9.34510
+-10.89639,-10.09428
+-9.43482,10.43549
+9.59539,10.04458
+9.29921,-10.69011
+-11.35667,-10.84220
+-10.34974,9.94992
+10.65653,9.95450
+9.35404,-8.71725
+-12.31798,-9.63993
+-10.11493,10.48078
+8.86091,9.51643
+8.52137,-11.76419
+-11.51060,-7.20961
+-9.14861,12.52825
+10.50402,9.76279
+10.27784,-9.53051
+-7.88081,-8.11381
+-8.94527,9.65693
+12.14168,9.74196
+10.46123,-9.79415
+-9.11234,-8.78123
+-10.19764,10.66898
+11.08275,9.28952
+10.88633,-10.10051
+-9.95064,-11.47337
+-9.77065,10.39936
+9.19293,10.08214
+10.00050,-10.36045
+-9.86822,-10.47391
+-11.63031,8.54495
+10.72030,9.32423
+10.50180,-11.28271
+-11.51735,-10.28547
+-12.10009,10.83531
+9.96113,10.58494
+9.13142,-9.95384
+-8.20845,-8.83636
+-10.13080,9.71941
+11.04008,9.07052
+8.98361,-9.56995
+-11.42050,-10.61993
+-11.04541,11.70623
+9.28612,9.36009
+9.60697,-10.72514
+-9.18971,-7.37746
+-10.83873,8.82995
+9.80222,9.44590
+9.25911,-10.55116
+-9.45939,-9.40700
+-12.15991,10.65348
+11.20617,11.69304
+8.82167,-10.04065
+-10.04456,-10.33913
+-10.99728,12.25851
+9.83275,10.08054
+10.61900,-9.59195
+-9.33261,-11.27423
+-9.89064,10.47202
+12.25094,10.40723
+9.37527,-9.07585
+-12.48904,-10.53364
+-10.41903,14.37430
+9.05662,10.72076
+11.12268,-9.38201
+-10.61498,-12.36239
+-7.58043,9.94951
+11.16691,10.95880
+9.57452,-10.05964
+-11.21398,-10.49165
+-9.66177,9.77798
+10.68659,11.85321
+9.83498,-11.69408
+-10.92877,-12.99724
+-10.46568,9.55430
+9.07698,10.26717
+8.33220,-9.68943
+-8.99223,-11.60371
+-9.55914,9.37256
+9.29106,10.29467
+9.44959,-9.53147
+-9.09694,-10.65793
+-9.12003,9.64209
+10.65031,9.88383
+9.95267,-9.96985
+-12.22739,-10.49048
+-10.34011,10.76806
+9.17916,9.81711
+10.66263,-11.53420
+-10.20435,-11.12145
+-9.35568,10.12763
+10.44444,10.10804
+9.28407,-8.87370
+-11.96868,-11.07016
+-11.39374,8.65738
+10.42916,8.91125
+10.68614,-10.17897
+-9.14308,-12.14468
+-9.61975,10.62150
+10.42455,9.59605
+10.95173,-8.90432
+-11.34973,-8.72370
+-8.92818,10.23094
+11.72578,8.68598
+10.47247,-8.97414
+-8.97748,-8.80182
+-11.27674,9.40846
+8.45452,10.79057
+9.35670,-9.52918
+-8.57810,-9.42318
+-10.22661,10.29577
+9.16554,10.22128
+9.49945,-9.96840
+-9.98503,-9.54943
+-8.58208,9.52524
+7.97510,11.13887
+9.90756,-9.66856
+-9.91555,-9.66258
+-9.23344,10.30881
+8.55872,10.48430
+9.69578,-8.77549
+-10.15075,-9.20985
+-9.23989,9.70880
+8.95468,9.90988
+11.26320,-9.11246
+-9.90931,-10.83343
+-8.30302,11.18158
+10.53516,10.17694
+11.58383,-9.90743
+-8.96557,-10.82538
+-9.21534,9.58231
+10.97234,10.96831
+9.58968,-11.39518
+-8.78821,-9.69662
+-10.07346,10.07969
+9.21805,9.60079
+9.90884,-10.95530
+-10.50627,-10.94619
+-9.94579,10.81070
+9.01830,10.45681
+8.82160,-10.15504
+-9.87976,-10.62050
+-9.80744,9.44024
+9.00543,9.55885
+9.59646,-9.47821
+-9.09088,-10.53221
+-11.34479,8.43459
+9.31827,9.58953
+11.19461,-8.40723
+-8.95691,-9.17418
+-9.75157,8.57524
+10.36600,10.61733
+9.04061,-10.18113
+-11.22210,-7.76176
+-9.45393,8.62093
+9.54483,11.96758
+10.41973,-8.75075
+-9.54946,-9.79222
+-11.18257,9.43834
+10.71380,10.68247
+11.94149,-11.29073
+-7.97783,-10.30837
+-11.10647,10.50398
+10.47389,8.86730
+8.99667,-11.09255
+-10.61512,-9.72604
+-9.29470,11.28986
+9.60972,9.58863
+10.16367,-10.52496
+-9.61994,-8.88012
+-11.03740,9.86769
+8.00070,11.27148
+9.25333,-9.01271
+-12.29877,-9.61899
+-9.87670,10.29582
+11.29260,9.92680
+10.54463,-8.60377
+-10.36759,-10.74966
+-7.37443,10.32432
+8.34657,9.23099
+9.35033,-10.76566
+-9.78956,-9.96384
+-9.17763,9.21174
+10.12714,9.83091
+10.27372,-9.80702
+-9.77715,-9.69185
+-10.62993,10.22723
+9.27382,11.73390
+11.10896,-10.89370
+-8.93518,-9.83411
+-7.85170,10.47639
+10.54197,9.35442
+10.62469,-10.25284
+-10.65074,-11.61832
+-10.26255,9.18022
+10.76485,9.36196
+9.69993,-9.32928
+-10.84704,-9.33400
+-10.35437,10.84954
+9.52064,10.10531
+10.86717,-9.30601
+-8.46116,-8.81039
+-10.70768,8.39708
+7.73748,11.74995
+11.33867,-8.76204
+-10.59186,-11.85749
+-9.12263,8.64732
+10.18344,10.29148
+11.03035,-10.76151
+-9.12266,-10.37320
+-11.08303,9.91925
+8.95440,9.48838
+11.65814,-11.04099
+-8.27198,-10.13045
+-9.24560,9.27741
+10.85228,8.79297
+11.18147,-9.59805
+-10.30414,-9.32990
+-10.11531,11.32240
+10.17672,9.04616
+10.05458,-9.71534
+-10.63690,-10.32852
+-9.81478,10.02603
+11.07705,9.67279
+9.00084,-10.20812
+-10.92354,-10.12149
+-9.66097,11.41678
+9.47680,9.48748
+11.69859,-7.97464
+-11.94734,-9.26027
+-10.46678,9.93762
+11.13101,9.77925
+9.40342,-11.52710
+-10.70480,-10.80311
+-9.57340,7.94487
+8.83283,11.72784
+9.29445,-10.02660
+-10.66090,-9.30256
+-10.51646,11.26170
+10.74992,9.54282
+8.12960,-10.48756
+-8.62500,-10.99914
+-11.15364,9.53803
+10.57310,10.52390
+11.00782,-9.21989
+-9.90136,-8.42091
+-9.91969,10.90039
+9.62977,10.92802
+9.60838,-10.19850
+-12.10327,-8.94986
+-9.77750,10.11199
+10.80321,11.48623
+9.55985,-10.11750
+-10.98474,-11.22607
+-9.77395,10.12129
+7.13193,10.36159
+10.86606,-10.21109
+-9.70367,-9.01890
+-11.99932,9.71503
+9.53346,9.23632
+10.50164,-8.45089
+-11.70071,-12.41913
+-9.05213,9.45728
+10.27459,10.68122
+11.01455,-9.19931
+-10.46934,-9.54741
+-10.38312,12.08343
+9.83500,9.44202
+9.01053,-9.94186
+-11.87975,-9.72556
+-9.45500,10.57025
+7.83587,9.27006
+11.23979,-8.44989
+-8.54203,-11.73485
+-10.49022,9.48235
+10.16762,8.80879
+10.01355,-11.43191
+-8.89205,-9.94454
+-9.99853,10.30713
+11.63704,9.82996
+10.60718,-9.68903
+-8.96955,-9.36368
+-11.23806,9.92298
+10.50283,10.35444
+9.63515,-10.50261
+-9.13634,-11.49938
+-9.67054,9.09565
+8.96358,9.05174
+11.28867,-10.26572
+-9.36235,-8.66685
+-10.84116,11.03214
+8.36523,10.35548
+11.71435,-7.05951
+-10.05224,-9.60214
+-9.32288,9.76678
+9.10702,10.04880
+9.24436,-8.99886
+-10.24698,-9.95568
+-9.38030,10.26894
+9.89760,9.44315
+10.87138,-9.43378
+-10.05854,-9.22123
+-9.19845,9.97470
+9.28045,11.31918
+9.63477,-8.89533
+-10.55592,-9.52398
+-10.57039,8.96571
+8.44187,8.50743
+8.41077,-9.60214
+-10.10754,-10.01304
+-8.70487,10.01547
+10.01774,9.40519
+9.51957,-10.40959
+-9.40900,-9.77395
+-10.04858,10.71705
+11.82003,10.83871
+10.91539,-10.57170
+-11.35081,-9.43172
+-10.52442,11.80326
+9.26636,11.71687
+10.03680,-10.06553
+-9.76171,-9.46236
+-7.76278,8.80364
+10.68363,11.24594
+10.69988,-9.21514
+-10.13488,-9.38438
+-11.45700,9.50165
+9.92928,11.27448
+9.34674,-10.69581
+-9.06847,-8.79462
+-9.46074,11.75493
+9.89226,9.65058
+10.32365,-9.22863
+-10.58998,-10.63546
+-8.51909,10.13368
+9.98844,9.39482
+10.78240,-9.44385
+-9.42046,-9.44522
+-10.32627,10.81075
+10.20170,9.28505
+10.04136,-8.18075
+-8.64815,-10.62449
+-10.63510,12.98990
+10.04402,9.84854
+9.82032,-9.02223
+-10.01476,-8.26745
+-9.54542,10.36398
+9.66283,10.89788
+9.50532,-10.62149
+-10.75558,-9.70852
+-9.10980,10.12722
+10.66148,9.86315
+9.48110,-10.06901
+-12.14414,-10.42575
+-8.73415,9.40110
+8.81856,10.62652
+10.34405,-9.16037
+-10.43047,-9.89322
+-9.71400,10.16967
+10.87417,9.24859
+11.13091,-12.04375
+-8.17256,-9.79836
+-12.28258,11.11156
+10.72291,10.94298
+9.95308,-9.86910
+-9.26449,-9.49323
+-8.00621,9.62550
+11.94165,9.98033
+8.78422,-9.65283
+-10.17497,-10.05317
+-11.15636,8.97802
+10.20428,9.99690
+9.14880,-10.97491
+-10.42889,-10.26683
+-8.97683,10.07680
+10.15468,10.94610
+10.14935,-8.44245
+-9.68732,-10.24960
+-8.40231,9.65178
+8.89483,10.88020
+11.21369,-10.07336
+-10.74045,-10.43951
+-9.56490,8.87484
+10.13536,9.41572
+8.66720,-9.68314
+-10.56826,-10.34859
+-9.13841,11.08562
+11.80313,10.00202
+10.89037,-8.65237
+-10.41239,-8.73954
+-11.52444,10.62874
+9.43685,11.05972
+9.47490,-11.33810
+-9.43984,-8.80979
+-9.01479,10.66674
+8.26144,9.71544
+9.29307,-10.81216
+-8.98165,-10.76464
+-10.37910,10.01031
+10.54509,11.03027
+10.19993,-9.17085
+-11.45139,-10.41297
+-8.55161,9.38685
+8.79649,10.39189
+10.13079,-11.06743
+-11.13432,-11.25358
+-9.44025,9.42151
+10.96495,9.15200
+10.67875,-7.27619
+-11.93976,-10.05915
+-11.34260,10.69225
+9.86075,9.74804
+11.44688,-9.32548
+-9.27377,-11.57196
+-8.28845,9.80364
+7.96326,8.30219
+10.26456,-9.14031
+-9.81497,-10.10009
+-8.47220,10.34172
+10.62875,10.52590
+10.19471,-10.29651
+-10.44057,-9.92252
+-11.60176,9.98474
+8.20600,9.66293
+10.38227,-8.75561
+-10.59001,-8.87722
+-10.10297,10.06496
+11.07588,8.28580
+11.02722,-9.71153
+-11.18406,-9.51660
+-9.42009,11.96196
+8.93700,9.20869
+10.50538,-11.32212
+-11.02504,-11.06982
+-9.91285,10.22943
+9.49799,10.12384
+8.91510,-10.61414
+-9.78045,-9.43778
+-10.64781,9.74714
+9.37083,8.82195
+8.99463,-11.23067
+-9.16965,-7.87183
+-10.07458,9.81087
+11.31121,9.87212
+9.33688,-8.66995
+-9.49440,-8.58367
+-10.13976,9.79716
+9.62426,8.76776
+8.56647,-8.94211
+-8.50809,-11.09994
+-9.66308,9.35997
+9.24669,10.00035
+12.01419,-9.36291
+-10.85837,-9.36247
+-8.81797,9.86255
+9.63060,10.59925
+9.94340,-10.77212
+-9.64772,-9.85050
+-9.72294,10.04734
+10.75424,10.44971
+8.52219,-10.28140
+-11.04242,-10.05440
+-10.21177,8.85792
+10.44920,10.01267
+9.57018,-10.30161
+-7.90367,-8.51750
+-10.17735,8.30380
+10.39862,11.16753
+9.08796,-9.76918
+-10.16761,-9.27390
+-9.04620,10.06582
+8.91328,8.55131
+11.73536,-7.61446
+-9.76887,-9.22034
+-9.08596,10.45261
+8.33021,9.58962
+11.11613,-10.49754
+-10.86122,-9.59865
+-9.20790,9.41802
+10.05432,11.69126
+8.99151,-10.54844
+-11.10970,-10.94226
+-9.42715,10.12651
+11.81416,9.58778
+9.93956,-8.78619
+-8.52433,-12.17764
+-9.30084,11.37267
+11.55290,8.59976
+9.36663,-8.86886
+-8.84726,-9.12492
+-7.25605,9.05708
+9.75674,10.28158
+11.27361,-11.46179
+-10.93185,-8.75034
+-9.56015,11.28908
+10.44080,9.56648
+10.29213,-10.04604
+-8.85609,-9.46437
+-9.02795,9.44686
+9.53790,8.83949
+9.59953,-9.14937
+-10.89206,-9.47454
+-8.93375,10.38803
+9.31406,10.02553
+10.36403,-10.51389
+-10.83184,-10.23732
+-10.67032,10.14746
+10.26527,10.58768
+9.50936,-10.35898
+-10.00133,-10.91798
+-9.97141,11.94514
+11.77083,7.79901
+10.82858,-8.85397
+-10.47690,-10.41646
+-11.08373,9.56660
+10.01902,10.59251
+9.20019,-9.77382
+-11.10100,-10.51354
+-10.17684,9.39770
+9.54912,10.04781
+10.87612,-11.67873
+-11.06904,-10.50736
+-9.54211,11.05013
+10.43618,11.25707
+12.57111,-10.70985
+-10.19524,-9.76111
+-8.83119,9.74598
+11.13779,11.66050
+11.44545,-10.50434
+-10.97061,-11.66460
+-11.80541,9.73520
+9.54706,12.33711
+8.78132,-9.58025
+-11.82834,-8.09835
+-10.06321,9.89012
+8.78789,11.16422
+9.08677,-9.56383
+-10.23806,-7.85366
+-9.07155,9.08045
+10.86618,10.68632
+10.45065,-10.89149
+-9.38851,-10.65349
+-9.20635,9.00525
+11.00138,9.81924
+9.43174,-11.39472
+-7.83875,-10.90479
+-8.31226,8.56361
+11.39410,10.53969
+11.06898,-10.06352
+-10.10202,-11.12941
+-9.55995,11.12729
+10.74775,11.70272
+9.21358,-10.57270
+-11.55430,-7.88275
+-9.68261,9.92432
+11.32704,9.23834
+11.14408,-10.12543
+-10.22918,-9.96844
+-11.02575,10.88486
+10.41381,10.16912
+10.05575,-10.00368
+-10.88382,-8.21916
+-11.28130,9.54441
+9.29477,10.53630
+9.28946,-10.93963
+-9.64020,-8.71782
+-10.82399,9.18348
+9.46809,10.82917
+7.93203,-10.66871
+-9.41398,-11.69381
+-12.10029,8.91736
+8.82729,9.46887
+11.16079,-9.08464
+-9.75404,-9.65077
+-7.66661,10.82419
+9.24632,8.51786
+12.01878,-9.80085
+-10.35618,-9.70357
+-9.95358,10.75211
+10.68472,11.61552
+8.30664,-9.44891
+-8.48368,-9.27400
+-8.80974,9.30184
+9.85184,11.73324
+9.52816,-10.45304
+-9.46493,-7.51715
+-8.83306,10.83810
+9.18891,12.26943
+11.15113,-10.16009
+-10.16778,-9.82849
+-8.93481,11.12711
+9.56556,10.18716
+11.12026,-10.24787
+-10.09236,-9.29725
+-9.62331,9.67847
+9.80809,8.96492
+9.83242,-9.81414
+-10.75180,-9.41800
+-9.24822,9.39906
+10.17271,9.70025
+8.33239,-9.50814
+-8.19927,-10.14601
+-8.88647,11.13711
+8.96833,10.66376
+9.46370,-9.69234
+-10.29161,-7.93794
+-10.76615,9.67304
+9.27667,9.62683
+9.04373,-11.96661
+-9.54254,-9.42985
+-10.36212,9.46539
+9.82709,8.34038
+9.54062,-9.83993
+-10.69059,-10.12989
+-8.98771,10.58693
+9.55778,9.51953
+8.96593,-9.51302
+-9.53975,-9.08816
+-8.54535,8.58736
+10.71401,10.46885
+10.37816,-11.47808
+-9.22629,-10.46460
+-11.45478,7.93188
+9.69992,11.01392
+10.67616,-10.33579
+-10.88400,-10.93487
+-8.86945,10.42734
+9.97082,9.66357
+9.59081,-10.14693
+-10.34222,-10.62567
+-8.06575,9.42486
+10.72420,8.60327
+9.13441,-8.82108
+-8.45596,-10.02486
+-9.42430,9.25398
+9.63041,9.97652
+10.26825,-10.75241
+-11.01394,-10.22112
+-9.60935,8.46379
+11.17461,10.38841
+9.72522,-10.14409
+-9.29724,-9.81642
+-10.96262,8.93812
+9.32127,10.36968
+9.21892,-11.10634
+-8.54363,-8.56733
+-9.00826,11.96452
+10.31989,8.67734
+9.36705,-9.87524
+-9.79035,-8.64544
+-9.07211,8.87182
+11.41126,9.25786
+10.32338,-8.51890
+-9.31112,-9.64424
+-10.73529,10.40711
+9.05669,9.32674
+11.02417,-10.34283
+-10.04618,-9.30187
+-10.14920,9.83711
+9.54411,10.86166
+10.66644,-12.05970
+-8.90298,-11.37789
+-10.68063,10.41681
+9.33948,7.86214
+10.18008,-8.31869
+-9.08507,-9.88131
+-11.67766,10.39028
+9.98141,10.08348
+10.59319,-10.79963
+-11.05074,-9.29838
+-9.71998,10.06996
+8.27469,10.37690
+9.40538,-8.18616
+-11.00967,-9.38963
+-9.86460,10.68216
+9.92323,8.89432
+10.61973,-10.72434
+-9.83042,-10.33042
+-9.77191,10.76787
+10.55179,7.70528
+11.37193,-10.30216
+-10.10887,-9.89974
+-9.39823,12.01322
+11.29586,8.06012
+8.49265,-10.12116
+-10.18969,-10.41766
+-10.06669,9.16949
+10.39427,7.26377
+9.91064,-10.95119
+-11.49367,-11.62634
+-10.56707,9.81848
+9.05843,10.14562
+9.81423,-10.46265
+-9.52590,-10.72658
+-10.07306,8.34089
+9.47539,10.78907
+8.65621,-9.89627
+-10.61017,-10.87395
+-10.99023,12.43746
+11.60002,9.91771
+10.35375,-9.02340
+-11.54866,-8.84185
+-9.22172,7.65296
+9.36445,8.74718
+9.87283,-10.48263
+-11.25606,-8.55961
+-9.75511,10.76072
+9.28566,10.91664
+10.79001,-10.42945
+-9.85105,-7.97608
+-10.03493,10.15828
+10.40980,10.71825
+9.12379,-7.72928
+-9.81120,-9.96665
+-8.17459,8.80512
+10.43266,11.01530
+8.46927,-10.32749
+-10.01577,-9.70868
+-11.13137,10.44525
+11.10222,8.56174
+12.06337,-10.63114
+-12.10071,-11.42622
+-9.42736,9.83330
+10.78456,10.99827
+10.26007,-8.24511
+-9.55499,-10.27783
+-8.53978,9.32006
+9.02309,8.79162
+10.63655,-9.91978
+-10.75552,-10.33800
+-11.03576,9.12427
+10.63681,9.38161
+10.73743,-11.64618
+-10.36248,-9.17506
+-9.30212,8.28921
+9.06708,11.34316
+9.75904,-10.40403
+-11.26427,-9.40368
+-10.55563,11.00849
+9.18776,9.22519
+10.21502,-11.70091
+-9.95581,-11.18518
+-10.53130,11.14803
+8.55297,9.77966
+10.27206,-11.70328
+-8.80493,-11.15871
+-9.19544,10.35317
+9.96755,9.16877
+9.77941,-8.56784
+-8.33319,-7.89772
+-11.82529,7.38231
+9.49318,9.19640
+9.13133,-7.90163
+-11.38590,-12.05326
+-10.85406,10.94556
+9.73146,8.99219
+9.34067,-9.00330
+-10.54708,-9.72197
+-9.45592,8.90686
+8.34307,9.05504
+8.20782,-11.03500
+-11.14902,-10.75966
+-11.20747,9.38150
+10.39910,9.06560
+11.90709,-11.16561
+-10.59856,-10.03603
+-9.81535,11.56075
+9.66915,11.76385
+10.03986,-10.56626
+-10.77348,-9.92614
+-10.18948,11.04791
+11.68160,9.26757
+10.54828,-9.60571
+-11.18654,-10.51388
+-10.34841,11.01230
+9.86607,8.99158
+9.00403,-10.33487
+-13.02546,-9.46392
+-8.64697,8.98336
+10.41814,9.06647
+10.19408,-7.50915
+-8.81651,-11.10234
+-10.21699,11.70836
+8.64747,9.95179
+10.49822,-9.12993
+-10.57740,-10.73866
+-9.40933,8.16698
+9.36936,10.20841
+10.98631,-8.00306
+-10.78102,-9.37814
+-9.85619,9.74531
+10.77051,9.82980
+10.69787,-10.85501
+-11.23334,-9.29526
+-8.28492,10.87849
+9.51642,10.36044
+7.90196,-9.45301
+-10.20893,-6.26850
+-10.12105,9.97696
+9.18078,10.82920
+8.87301,-9.86775
+-11.02197,-11.10711
+-12.09659,10.23905
+9.35954,9.57097
+9.72401,-10.78819
+-9.10859,-11.36901
+-9.79348,12.05160
+10.58234,10.86335
+11.31912,-10.58772
+-9.22449,-10.09432
+-10.70947,9.82765
+9.03554,9.22072
+8.04264,-11.13687
+-10.01358,-11.15447
+-8.81941,11.36654
+8.25005,10.26521
+9.52412,-9.61044
+-9.97422,-9.53162
+-10.30825,10.90167
+10.44285,10.30996
+10.30283,-10.02560
+-9.04249,-12.28448
+-11.93813,8.13346
+9.82128,10.50389
+8.81351,-9.37827
+-10.79956,-10.88787
+-8.20262,10.54611
+10.10659,9.07897
+9.56526,-7.95821
+-10.27213,-9.86584
+-11.40688,11.81927
+10.45657,8.63612
+9.78308,-9.54856
+-8.72571,-10.21130
+-10.07903,9.63492
+9.77661,10.16328
+10.59173,-8.32068
+-9.64575,-9.55562
+-9.38771,12.08158
+9.93099,9.02490
+9.78081,-11.25844
+-8.56043,-10.81394
+-10.77139,9.60264
+10.60411,7.49190
+9.92656,-9.85208
+-10.28799,-9.62098
+-8.89787,9.57780
+9.43453,9.13274
+9.26295,-10.39941
+-10.53368,-10.63825
+-9.03186,9.89231
+10.26405,9.09905
+9.91932,-11.10842
+-10.92176,-10.59297
+-10.01661,9.07478
+11.48766,9.00724
+10.94380,-8.59833
+-9.98178,-10.30570
+-9.49889,11.55845
+9.24314,10.69098
+12.43193,-9.49512
+-11.25590,-11.18024
+-11.33259,11.28093
+11.30624,11.00516
+9.28116,-9.26294
+-9.08490,-10.97669
+-10.27947,10.83122
+9.96092,10.43359
+9.17805,-9.90727
+-10.82677,-10.01519
+-8.21937,9.96157
+9.37806,10.33027
+9.44095,-10.81034
+-11.13842,-8.17673
+-10.09406,11.00342
+8.13909,10.04268
+10.54064,-9.71542
+-8.69798,-9.91330
+-10.15179,12.01352
+8.44083,9.85149
+9.38906,-10.04835
+-11.99801,-10.57803
+-9.47248,11.33342
+10.42133,9.61736
+11.97842,-10.14070
+-11.04294,-10.59249
+-9.32539,10.36103
+9.72759,10.82919
+9.06217,-10.64002
+-8.80950,-9.87509
+-10.80158,11.58523
+11.36444,9.66316
+10.94493,-10.19247
+-9.40222,-9.59172
+-10.98992,10.37928
+9.95810,11.17053
+10.45520,-11.33156
+-11.74848,-12.60398
+-10.49334,9.86432
+9.56364,9.54618
+8.35082,-10.09109
+-9.75263,-9.26391
+-10.41458,10.95932
+9.01793,8.91708
+10.87523,-10.45733
+-9.55297,-11.84034
+-9.10251,9.93602
+9.97126,10.73513
+9.94328,-10.19465
+-8.55882,-9.52025
+-8.61371,9.38747
+10.27041,8.96418
+10.25433,-7.43296
+-8.74803,-9.69223
+-11.80975,10.44145
+8.85985,9.57394
+10.29571,-10.04947
+-11.10739,-8.35617
+-9.94659,11.39819
+8.30890,11.19616
+9.92803,-9.56333
+-9.44209,-10.22905
+-9.05097,10.20146
+9.82321,9.41887
+11.79490,-8.79085
+-9.75495,-9.40028
+-11.20484,10.56969
+9.58252,9.29431
+11.57830,-10.99745
+-9.26419,-10.21263
+-9.77350,10.34124
+8.86047,8.96034
+11.41108,-11.10050
+-8.89966,-9.53919
+-8.46375,11.28895
+10.67782,10.11867
+10.60849,-11.58210
+-10.72530,-8.96385
+-8.57572,10.16218
+9.89778,9.62236
+9.30185,-9.74896
+-9.24940,-9.89855
+-10.06652,7.66403
+9.37232,7.98961
+9.67042,-10.65211
+-8.08708,-9.92838
+-10.91895,9.99224
+8.73229,9.30918
+10.44166,-9.50905
+-11.93562,-9.19281
+-10.60197,8.99189
+10.11735,10.66241
+9.24944,-9.87193
+-9.58182,-10.01207
+-10.34833,11.21511
+9.67676,9.20808
+9.28688,-10.10452
+-8.74087,-11.00660
+-9.37814,9.62155
+9.63831,10.70814
+10.68929,-11.03016
+-10.96907,-9.99952
+-11.27823,9.83138
+11.35064,10.00418
+10.34518,-10.09881
+-8.14423,-9.41440
+-10.80597,9.41953
+10.17961,11.60660
+10.24417,-8.03396
+-11.06893,-10.62011
+-9.90176,9.19456
+9.26163,8.84776
+8.56471,-10.98725
+-9.65708,-9.61594
+-11.86609,9.99176
+8.28815,8.05475
+10.42551,-10.34158
+-10.28442,-11.50637
+-9.96584,11.53585
+8.50355,10.78609
+10.15939,-9.86958
+-11.35825,-10.60802
+-9.76281,10.77890
+10.82674,9.58541
+10.66744,-10.42090
+-10.06058,-9.51107
+-9.61539,10.18773
+9.65477,10.35701
+9.42670,-9.50163
+-8.70516,-9.03740
+-10.14817,9.89926
+8.60940,9.85848
+10.81443,-10.18702
+-10.89589,-8.84456
+-10.72961,10.30370
+11.48889,10.83805
+10.53722,-11.20210
+-8.89972,-8.87899
+-10.60640,9.60213
+8.79202,9.47705
+9.55456,-11.26679
+-9.72026,-10.26300
+-9.82860,9.66980
+9.64219,8.39072
+11.96873,-12.43930
+-12.69889,-10.45322
+-10.12154,9.62533
+8.87464,10.33099
+9.78567,-10.49582
+-10.97521,-10.22897
+-10.05204,8.78900
+9.13289,9.95104
+9.99961,-9.65208
+-11.32210,-10.99970
+-9.39506,11.06023
+10.69867,10.21707
+11.18530,-10.57605
+-11.36657,-10.66227
+-11.17960,10.63667
+10.05520,11.56174
+8.53644,-8.94009
+-9.86777,-10.60972
+-10.18327,11.36255
+9.66957,10.42676
+9.10339,-9.39690
+-10.19730,-10.07280
+-11.88853,10.07983
+9.61822,9.78342
+9.95505,-11.75699
+-10.50890,-9.69965
+-10.06463,8.87295
+11.52596,7.85585
+8.01283,-10.17466
+-8.97867,-8.01563
+-11.08742,9.06604
+10.65298,10.69547
+9.56609,-10.75591
+-11.82813,-11.93360
+-10.87935,11.18378
+9.48121,8.83344
+9.72927,-7.74412
+-10.15711,-9.38524
+-10.03047,9.71351
+11.73515,9.80973
+11.66222,-10.81710
+-8.22552,-8.72200
+-8.90344,9.89797
+9.75573,10.13477
+9.69232,-10.79772
+-10.98454,-11.75136
+-11.26148,11.01209
+11.19911,9.21648
+11.24773,-10.84740
+-9.34384,-9.95816
+-11.28451,10.22344
+10.42368,9.35696
+9.44563,-10.05020
+-8.92734,-11.44288
+-8.76003,9.60491
+8.31480,11.04765
+10.30023,-8.84046
+-10.69748,-9.69772
+-9.74686,9.91445
+9.06527,9.36174
+10.29706,-8.56935
+-9.12218,-9.84714
+-10.73886,9.69305
+10.69466,8.71945
+7.46673,-8.32557
+-10.46270,-9.43885
+-10.18831,11.92671
+9.54488,10.53606
+11.20279,-9.72198
+-10.29352,-9.41728
+-9.66132,9.46736
+9.55624,10.87939
+8.70647,-11.38569
+-9.31436,-10.76122
+-9.65285,10.53365
+11.46317,8.61904
+11.54334,-10.03296
+-8.86531,-8.11221
+-9.39053,9.03911
+8.95417,9.33732
+9.17753,-11.71715
+-9.73417,-9.95174
+-9.15066,10.88720
+9.90958,9.56333
+10.42534,-10.64452
+-10.54098,-11.16846
+-9.74910,10.33209
+10.17993,9.83010
+8.96152,-9.33190
+-10.42903,-9.80034
+-9.62769,11.01596
+11.08516,11.18930
+10.77089,-8.32920
+-10.13765,-10.78928
+-9.26632,8.68140
+10.26152,9.70587
+8.91066,-10.08641
+-8.90724,-10.20665
+-10.60906,8.76451
+9.10452,9.35750
+9.61913,-11.04773
+-9.52042,-11.36316
+-9.12694,10.12537
+10.49400,10.94122
+10.37454,-9.17352
+-8.46962,-9.66007
+-9.21958,10.63248
+9.65815,10.24536
+9.66859,-10.39386
+-10.15341,-9.58978
+-8.81835,7.34989
+8.03391,9.41644
+7.63370,-9.46713
+-9.44574,-10.02151
+-10.91145,9.48780
+9.48062,10.26290
+11.50844,-10.11152
+-10.02875,-9.39516
+-10.77525,10.67458
+9.70218,8.96693
+9.14831,-9.58419
+-9.95330,-9.26713
+-7.91991,10.87652
+9.09581,9.69162
+10.56334,-10.57076
+-9.31916,-10.25477
+-8.86857,10.40518
+9.69131,11.22816
+10.43168,-9.43882
+-11.53927,-10.19593
+-11.65010,8.98661
+9.82068,9.75609
+10.42304,-10.63139
+-10.77796,-9.78363
+-10.14679,8.38838
+10.04846,11.44108
+8.91787,-11.84203
+-10.50152,-9.09624
+-9.23858,10.37673
+9.51988,9.19472
+9.62635,-10.98493
+-7.96832,-8.46967
+-9.23958,9.99432
+8.52600,10.49431
+10.45735,-8.95542
+-10.10670,-8.94483
+-12.04178,11.64896
+12.04985,13.19957
+10.11005,-9.80724
+-9.31272,-9.26809
+-10.19468,7.83123
+10.06583,9.83924
+10.37025,-10.70537
+-9.99045,-9.79576
+-11.60614,10.35134
+9.66911,10.20208
+6.75939,-9.85073
+-10.41799,-9.88503
+-9.27451,10.58633
+11.54579,9.64337
+10.86169,-10.10384
+-8.26402,-9.38054
+-9.84269,11.06383
+10.09015,9.80541
+9.60371,-10.25264
+-8.42722,-8.62865
+-9.47460,10.10157
+8.17999,10.55060
+10.91978,-8.97728
+-10.22564,-9.74968
+-9.59876,12.26474
+9.36060,11.71037
+9.06608,-10.04216
+-11.26228,-9.09473
+-12.64616,9.71632
+9.24412,10.97705
+9.95413,-9.87823
+-8.93546,-10.73928
+-10.83299,9.72572
+10.00305,8.97094
+11.07176,-10.29392
+-10.70769,-9.44726
+-10.81392,10.12652
+12.06773,8.79625
+10.87802,-10.64045
+-11.41413,-8.86397
+-11.17845,9.99102
+9.90578,11.44829
+8.81977,-10.25512
+-10.21971,-10.79532
+-10.32018,8.71796
+9.84632,10.26373
+10.28899,-8.87599
+-9.41846,-9.62188
+-10.87264,8.12494
+10.70972,10.47360
+10.46870,-11.06611
+-9.70998,-9.45777
+-11.22175,8.91312
+10.55425,11.57742
+10.21924,-10.37163
+-10.20753,-8.79023
+-10.14556,9.35359
+8.66518,12.12827
+11.19941,-9.74873
+-9.02804,-8.76458
+-8.74349,10.13198
+12.08948,10.71336
+12.11652,-9.52110
+-11.59153,-11.10086
+-10.28588,9.31274
+9.59185,9.42774
+11.78901,-8.74583
+-11.81790,-8.19799
+-11.27999,9.62383
+12.02474,10.38895
+10.01742,-9.18408
+-8.52993,-10.67305
+-8.73516,11.50941
+9.49379,9.94168
+9.00718,-9.17876
+-9.25899,-9.89284
+-10.92993,9.25848
+11.91958,10.68596
+10.43284,-8.86410
+-11.47375,-11.24818
+-12.55960,10.11561
+8.51306,10.07681
+10.83428,-9.01108
+-8.29667,-10.88084
+-10.30406,10.44005
+11.40166,10.20894
+10.70052,-9.66818
+-8.50384,-9.94887
+-10.66434,9.37087
+11.82635,9.81528
+10.87959,-10.94774
+-10.07738,-9.42964
+-9.52889,10.03914
+8.60898,10.10547
+11.07129,-8.18069
+-10.03131,-9.04796
+-9.46761,8.63736
+9.62742,12.80150
+11.02841,-10.31087
+-8.82817,-10.48651
+-9.76428,9.35888
+10.17720,8.68207
+11.37864,-11.22601
+-10.19777,-8.82008
+-7.27643,10.15895
+10.72909,9.60499
+11.24996,-8.56875
+-11.78686,-9.96212
+-10.60331,9.26126
+9.54832,12.87405
+10.49429,-10.22091
+-10.75411,-11.06200
+-10.58386,11.22221
+9.60659,11.74797
+9.40974,-11.02208
+-11.63573,-10.38777
+-10.65737,9.38417
+9.95714,8.54889
+8.44478,-9.89972
+-9.53768,-10.66541
+-9.60973,11.13657
+9.53843,10.48036
+9.38897,-10.17251
+-9.45254,-10.21658
+-8.07684,9.36163
+9.33984,11.26337
+9.41697,-9.97323
+-10.18730,-10.02043
+-9.53663,9.36985
+9.08210,9.93872
+10.74610,-8.91945
+-10.84760,-10.15216
+-9.79222,8.33868
+8.77394,9.70332
+9.50174,-9.52564
+-8.53745,-9.48858
+-9.27912,9.60562
+9.43167,8.42363
+11.00276,-9.31133
+-9.69775,-11.47921
+-11.27421,8.21176
+8.92453,10.53892
+9.12058,-9.64901
+-11.07594,-9.97660
+-7.63305,12.36342
+9.61158,9.68257
+9.51847,-10.22551
+-9.01187,-10.16904
+-9.17538,10.83855
+10.39562,7.06851
+10.42655,-9.18143
+-10.98659,-9.20219
+-11.05683,8.70404
+8.87202,8.73842
+9.37505,-9.84227
+-11.20224,-8.94569
+-10.61832,7.95198
+10.57393,11.20097
+10.31155,-11.07010
+-10.40719,-12.85651
+-9.03522,9.97637
+11.99483,9.70608
+7.99513,-11.83374
+-9.27248,-10.62570
+-9.19974,9.92116
+10.23994,10.85577
+9.71134,-8.76376
+-9.26000,-10.02374
+-10.15014,10.55904
+11.01457,10.21460
+10.75903,-10.68357
+-11.17585,-10.33189
+-11.37187,10.17871
+10.28873,10.11142
+8.65407,-9.49889
+-11.39079,-9.96078
+-9.47437,10.71722
+9.73846,11.03152
+9.65356,-8.92850
+-11.25337,-10.10890
+-11.34648,9.84868
+9.58547,11.11929
+11.70014,-10.91476
+-9.40814,-10.13850
+-11.37612,10.13873
+9.52032,10.63336
+7.49267,-9.47045
+-10.12527,-9.45950
+-10.13430,10.69618
+11.56469,12.64710
+9.51783,-10.38988
+-9.32104,-10.20453
+-9.47363,9.61569
+9.24755,9.54328
+10.07684,-10.50510
+-10.31747,-11.39775
+-9.39920,11.29358
+11.20372,9.42990
+8.58539,-9.45371
+-9.01566,-11.04251
+-9.95941,10.79309
+11.34069,9.55197
+8.94548,-10.63514
+-10.13000,-9.52564
+-11.36881,12.06095
+7.45036,10.22583
+10.35871,-8.63573
+-10.12560,-10.51399
+-9.07040,9.58662
+9.64396,8.42848
+10.75433,-9.51234
+-10.25018,-9.77820
+-10.53604,9.51042
+11.91496,10.20727
+9.22188,-9.86576
+-11.16985,-8.54255
+-11.86252,10.92041
+9.73020,11.18130
+9.52930,-11.13961
+-8.06880,-10.49418
+-10.47651,10.25719
+10.08479,8.67917
+9.54027,-11.06589
+-10.28179,-9.57857
+-10.33007,10.32182
+10.73695,9.34314
+9.98280,-9.98242
+-7.88805,-10.90772
+-9.92443,10.06591
+10.82983,10.89667
+11.40179,-10.57538
+-10.77811,-9.66939
+-8.97779,10.53787
+10.26757,9.12278
+9.72589,-8.91560
+-9.25753,-10.98277
+-9.58829,9.36151
+10.51050,10.29495
+9.20386,-8.67983
+-10.16154,-9.68441
+-10.40729,10.05248
+9.58260,11.34742
+8.81914,-9.50502
+-9.48057,-10.80522
+-7.97485,10.26650
+8.95138,9.91122
+9.21743,-9.70264
+-9.40240,-10.02634
+-10.73368,9.98672
+9.69560,9.99319
+9.92963,-9.89290
+-10.21272,-9.70848
+-10.35324,10.92246
+8.90710,11.58613
+9.74228,-7.43809
+-10.09742,-9.70007
+-10.66427,10.71937
+9.94247,9.05478
+10.48527,-10.53202
+-10.84938,-10.05108
+-9.40350,9.79718
+9.90010,11.01529
+9.42952,-9.97132
+-9.20481,-9.36427
+-10.64598,11.55268
+10.03317,10.94512
+10.82116,-10.13687
+-9.14400,-10.77046
+-10.72244,9.17250
+10.02543,11.47330
+11.59469,-9.74182
+-11.02035,-10.17359
+-9.35590,10.30453
+10.03638,10.84971
+9.42069,-11.13644
+-11.18468,-9.06012
+-10.21506,11.90395
+11.48561,10.71818
+10.85862,-11.21954
+-10.51411,-9.52978
+-10.44161,8.63244
+9.57370,10.58862
+9.01328,-11.00746
+-9.09768,-10.18923
+-11.37971,10.40840
+8.98015,10.28362
+10.53901,-9.88748
+-9.70126,-10.56936
+-9.70841,9.59371
+9.38814,9.03736
+9.66896,-10.22056
+-9.51194,-9.75199
+-8.56509,11.52717
+10.76441,10.71445
+9.70396,-9.87964
+-10.75548,-8.20699
+-9.74310,9.20169
+10.35346,9.67086
+9.59087,-9.85446
+-10.51283,-10.40976
+-8.13043,9.55418
+8.25953,9.97378
+9.63812,-10.91598
+-11.13718,-10.82271
+-9.28410,10.12813
+9.59464,10.22392
+11.02826,-9.81178
+-8.81843,-11.16361
+-10.52338,9.63700
+9.01835,9.46724
+7.99059,-9.93505
+-8.33463,-9.62910
+-11.15964,10.68302
+9.50097,8.42500
+9.55139,-10.88040
+-11.02752,-10.58390
+-8.36595,9.81303
+10.33483,9.86033
+10.06270,-10.44882
+-9.25175,-10.63886
+-8.68588,9.80741
+9.13862,11.34398
+8.97815,-9.93831
+-8.34438,-10.73496
+-11.26898,9.69849
+9.05554,10.69841
+8.41369,-9.14862
+-9.90580,-10.23952
+-9.95190,8.95961
+11.22735,9.89138
+9.79347,-9.19485
+-11.59843,-8.85735
+-10.31890,9.70064
+10.00842,9.24371
+9.94706,-10.84124
+-10.90041,-8.88167
+-8.91593,9.11032
+10.75228,12.43005
+9.60392,-9.89382
+-10.20011,-10.32675
+-8.73224,10.97639
+11.77200,8.56912
+6.87066,-9.19845
+-9.69998,-9.94592
+-9.81899,10.77205
+11.46439,9.72661
+9.87369,-9.99777
+-9.94667,-10.27202
+-9.74461,10.72289
+8.98990,8.51154
+9.25222,-10.28096
+-10.36732,-9.59699
+-9.81985,9.08795
+10.48662,11.96128
+10.54074,-11.37073
+-9.94617,-10.10699
+-9.19479,8.66002
+9.59150,9.96591
+8.34119,-10.63264
+-9.30508,-9.96232
+-10.38566,8.98055
+9.50754,9.02161
+9.04429,-12.06260
+-8.78943,-11.13960
+-7.90613,9.07015
+9.75007,10.76680
+8.91832,-10.89008
+-11.15594,-9.67158
+-10.79374,9.56286
+10.90234,10.15391
+9.02187,-9.44196
+-10.94407,-10.23002
+-8.20675,11.60524
+11.92994,10.39589
+9.43725,-10.83511
+-8.75376,-10.66920
+-12.39887,9.99473
+9.17577,9.36450
+10.20644,-9.62693
+-9.33647,-10.69703
+-8.07500,10.33085
+9.23491,10.08131
+10.55977,-9.49873
+-9.59453,-9.21983
+-9.55584,11.29457
+9.19292,10.75661
+10.97044,-10.12806
+-9.72559,-9.30842
+-9.07467,10.13695
+9.81501,10.46223
+10.76754,-8.69469
+-9.81130,-10.49711
+-10.81387,11.17711
+9.72410,9.31749
+9.37930,-10.22278
+-10.91102,-9.80647
+-12.00669,11.26946
+8.46830,9.46813
+8.98143,-10.23918
+-10.06189,-11.50411
+-10.06606,10.82325
+9.03003,9.80317
+9.75746,-9.95324
+-8.19617,-10.24170
+-10.96334,10.77782
+8.52488,11.98925
+10.35299,-11.54337
+-8.10840,-10.54457
+-10.96783,10.11619
+10.25229,7.36800
+9.63256,-10.59517
+-9.87104,-10.21122
+-9.45266,10.46668
+10.08752,7.73846
+11.59685,-9.03435
+-10.56355,-10.31182
+-10.27843,10.98991
+9.17214,9.72969
+8.96129,-8.66636
+-9.32861,-10.38775
+-7.90122,10.33753
+9.74726,10.18574
+8.44785,-10.79396
+-10.46224,-9.66036
+-8.43850,9.88322
+11.48458,11.68334
+9.11234,-8.29942
+-10.85115,-9.21978
+-8.39222,9.29802
+11.09416,7.54221
+8.37049,-10.10626
+-9.75918,-10.35592
+-8.54889,10.30682
+10.15230,10.64334
+9.02139,-12.02470
+-10.71457,-9.34875
+-10.46772,11.21123
+8.74883,10.47102
+9.82408,-10.19886
+-10.21639,-11.74068
+-9.93626,9.84448
+9.53694,9.44116
+11.57437,-10.74262
+-8.43746,-10.69760
+-10.22255,10.69340
+9.34809,8.92875
+10.86133,-11.27230
+-8.37569,-10.00130
+-10.87850,9.97969
+8.79732,11.91116
+10.54934,-10.41831
+-11.14232,-10.95648
+-9.62968,9.15550
+11.34296,8.77159
+9.39884,-9.65123
+-8.02237,-9.37117
+-10.19041,10.32526
+11.14107,10.79634
+10.89127,-8.62508
+-11.15667,-11.48816
+-10.30775,9.69569
+10.66592,9.23568
+10.61065,-9.49155
+-10.95378,-9.87249
+-8.58628,9.42066
+11.49711,9.89893
+9.64375,-9.64238
+-11.20614,-8.92329
+-9.77477,10.33345
+9.98465,10.34082
+9.17337,-8.74085
+-9.69137,-9.99914
+-10.68715,8.10400
+10.30512,9.04388
+9.29011,-9.40827
+-8.57255,-12.03453
+-10.92822,12.10141
+10.21442,11.02841
+9.92336,-9.74111
+-9.90231,-12.40773
+-10.19916,12.43700
+10.24949,9.64878
+10.75048,-10.47595
+-8.44590,-9.80523
+-11.45741,8.91979
+12.04646,12.05513
+11.13979,-11.05753
+-10.75619,-8.20706
+-9.11973,9.53412
+9.62255,10.91136
+10.89917,-8.62748
+-10.89013,-10.09799
+-10.66178,9.02398
+10.84863,11.25160
+10.63695,-11.83712
+-8.23128,-11.32812
+-11.21354,12.20318
+9.61992,8.95818
+9.72363,-11.53043
+-9.41976,-9.13998
+-10.77007,11.45873
+9.83207,11.12311
+8.79818,-11.65023
diff --git a/wayang-benchmark/src/test/resources/rheem.properties b/wayang-benchmark/src/test/resources/rheem.properties
new file mode 100644
index 0000000..736e864
--- /dev/null
+++ b/wayang-benchmark/src/test/resources/rheem.properties
@@ -0,0 +1,19 @@
+#
+# 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.
+#
+
+spark.driver.host = localhost
diff --git a/wayang-benchmark/src/test/scala/org/apache/wayang/apps/kmeans/KmeansTest.scala b/wayang-benchmark/src/test/scala/org/apache/wayang/apps/kmeans/KmeansTest.scala
new file mode 100644
index 0000000..53d6fea
--- /dev/null
+++ b/wayang-benchmark/src/test/scala/org/apache/wayang/apps/kmeans/KmeansTest.scala
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.kmeans
+
+import org.apache.wayang.commons.util.profiledb.model.{Experiment, Subject}
+import org.junit.Assert._
+import org.junit.Test
+import org.apache.wayang.core.api.Configuration
+import org.apache.wayang.java.Java
+import org.apache.wayang.spark.Spark
+
+/**
+  * Test suite for [[Kmeans]].
+  */
+class KmeansTest {
+
+  implicit val experiment = new Experiment("test", new Subject("test", "23.42"))
+
+  implicit val configuration = new Configuration
+
+  private def getTestFileUrl(fileName: String) =
+    Thread.currentThread().getContextClassLoader.getResource(fileName).toString
+
+
+  @Test
+  def shouldWorkWithJava() = {
+    val kmeans = new Kmeans(Java.basicPlugin)
+    val centroids = kmeans(
+      k = 4,
+      inputFile = getTestFileUrl("kmeans-k4-10000.input"),
+      iterations = 100,
+      isResurrect = true
+    )
+
+    assertEquals(4, centroids.size)
+    //    List(Point(-10, -10), Point(10, -10), Point(-10, 10), Point(10, 10)).foreach { expectedCentroid =>
+    //      assertTrue(
+    //        s"None of $centroids matches the expected centroid $expectedCentroid.",
+    //        centroids.exists(centroid => centroid.distanceTo(expectedCentroid) < 6))
+    //    }
+  }
+
+  @Test
+  def shouldWorkWithSpark() = {
+    val kmeans = new Kmeans(Spark.basicPlugin)
+    val centroids = kmeans(
+      k = 4,
+      inputFile = getTestFileUrl("kmeans-k4-10000.input"),
+      iterations = 100,
+      isResurrect = true
+    )
+
+    assertEquals(4, centroids.size)
+    //    List(Point(-10, -10), Point(10, -10), Point(-10, 10), Point(10, 10)).foreach { expectedCentroid =>
+    //      assertTrue(
+    //        s"None of $centroids matches the expected centroid $expectedCentroid.",
+    //        centroids.exists(centroid => centroid.distanceTo(expectedCentroid) < 6))
+    //    }
+  }
+
+  @Test
+  def shouldWorkWithJavaAndSpark() = {
+    val kmeans = new Kmeans(Java.basicPlugin, Spark.basicPlugin)
+    val centroids = kmeans(
+      k = 4,
+      inputFile = getTestFileUrl("kmeans-k4-10000.input"),
+      iterations = 100,
+      isResurrect = true
+    )
+
+    assertEquals(4, centroids.size)
+    //    List(Point(-10, -10), Point(10, -10), Point(-10, 10), Point(10, 10)).foreach { expectedCentroid =>
+    //      assertTrue(
+    //        s"None of $centroids matches the expected centroid $expectedCentroid.",
+    //        centroids.exists(centroid => centroid.distanceTo(expectedCentroid) < 6))
+    //    }
+  }
+}
+
diff --git a/wayang-benchmark/src/test/scala/org/apache/wayang/apps/util/ParametersTest.scala b/wayang-benchmark/src/test/scala/org/apache/wayang/apps/util/ParametersTest.scala
new file mode 100644
index 0000000..cf40930
--- /dev/null
+++ b/wayang-benchmark/src/test/scala/org/apache/wayang/apps/util/ParametersTest.scala
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.wayang.apps.util
+
+import org.apache.wayang.commons.util.profiledb.model.Experiment
+import org.junit.{Assert, Test}
+
+/**
+  * Test suite for the [[Parameters]].
+  */
+class ParametersTest {
+
+  @Test
+  def testCreateExperimentFull(): Unit = {
+    val experiment = Parameters.createExperiment("exp(id;tags=tag1,tag2;conf=a:1,b:true)", TestExperimentDescriptor)
+
+    val expectation = new Experiment("id", TestExperimentDescriptor.createSubject, experiment.getStartTime, "tag1", "tag2")
+    expectation.getSubject.addConfiguration("a", 1)
+    expectation.getSubject.addConfiguration("b", true)
+
+    Assert.assertEquals(expectation, experiment)
+  }
+
+  @Test
+  def testCreateExperimentMinimal(): Unit = {
+    val experiment = Parameters.createExperiment("exp(id)", TestExperimentDescriptor)
+
+    val expectation = new Experiment("id", TestExperimentDescriptor.createSubject, experiment.getStartTime)
+
+    Assert.assertEquals(expectation, experiment)
+  }
+
+  @Test
+  def testCreateExperimentWithTags(): Unit = {
+    val experiment = Parameters.createExperiment("exp(id;tags=tag1,tag2)", TestExperimentDescriptor)
+
+    val expectation = new Experiment("id", TestExperimentDescriptor.createSubject, experiment.getStartTime, "tag1", "tag2")
+
+    Assert.assertEquals(expectation, experiment)
+  }
+
+  @Test
+  def testCreateExperimentWithConf(): Unit = {
+    val experiment = Parameters.createExperiment("exp(id;conf=myLong:-234L,myDouble:-23.42)", TestExperimentDescriptor)
+
+    val expectation = new Experiment("id", TestExperimentDescriptor.createSubject, experiment.getStartTime)
+    expectation.getSubject.addConfiguration("myLong", -234L)
+    expectation.getSubject.addConfiguration("myDouble", -23.42d)
+
+    Assert.assertEquals(expectation, experiment)
+  }
+
+  @Test
+  def testParseAny(): Unit = {
+    Assert.assertEquals(-234L, Parameters.parseAny("-234L"))
+    Assert.assertEquals(-234, Parameters.parseAny("-234"))
+    Assert.assertEquals(0, Parameters.parseAny("0"))
+    Assert.assertEquals(23d, Parameters.parseAny("23."))
+    Assert.assertEquals(0d, Parameters.parseAny("0."))
+    Assert.assertEquals(-32.23d, Parameters.parseAny("-32.23"))
+    Assert.assertEquals(true, Parameters.parseAny("true"))
+    Assert.assertEquals(false, Parameters.parseAny("false"))
+    Assert.assertEquals(null, Parameters.parseAny("null"))
+    Assert.assertEquals("astring", Parameters.parseAny("astring"))
+  }
+
+
+  object TestExperimentDescriptor extends ExperimentDescriptor {
+
+    override def version: String = "1.0"
+  }
+}
diff --git a/wayang-commons/wayang-basic/src/test/java/org/apache/wayang/basic/operators/TextFileSourceTest.java b/wayang-commons/wayang-basic/src/test/java/org/apache/wayang/basic/operators/TextFileSourceTest.java
index 9b2e305..909169c 100644
--- a/wayang-commons/wayang-basic/src/test/java/org/apache/wayang/basic/operators/TextFileSourceTest.java
+++ b/wayang-commons/wayang-basic/src/test/java/org/apache/wayang/basic/operators/TextFileSourceTest.java
@@ -60,7 +60,7 @@
         when(optimizationContext.getJob()).thenReturn(job);
         when(job.getStopWatch()).thenReturn(new StopWatch(new Experiment("mock", new Subject("mock", "mock"))));
         when(optimizationContext.getConfiguration()).thenReturn(new Configuration());
-        final URL testFile = this.getClass().getResource("/ulysses.txt");
+        final URL testFile = this.getClass().getResource("/ulysses.input");
         final TextFileSource textFileSource = new TextFileSource(testFile.toString());
 
         final BufferedReader bufferedReader = new BufferedReader(
diff --git a/wayang-commons/wayang-basic/src/test/resources/ulysses.txt b/wayang-commons/wayang-basic/src/test/resources/ulysses.input
similarity index 100%
rename from wayang-commons/wayang-basic/src/test/resources/ulysses.txt
rename to wayang-commons/wayang-basic/src/test/resources/ulysses.input
diff --git a/wayang-platforms/wayang-giraph/src/test/java/org/apache/wayang/giraph/operators/GiraphPagaRankOperatorTest.java b/wayang-platforms/wayang-giraph/src/test/java/org/apache/wayang/giraph/operators/GiraphPagaRankOperatorTest.java
index cfe177e..436c711 100644
--- a/wayang-platforms/wayang-giraph/src/test/java/org/apache/wayang/giraph/operators/GiraphPagaRankOperatorTest.java
+++ b/wayang-platforms/wayang-giraph/src/test/java/org/apache/wayang/giraph/operators/GiraphPagaRankOperatorTest.java
@@ -68,7 +68,7 @@
         FileChannel.Instance inputChannelInstance =
                 (FileChannel.Instance) new FileChannel(FileChannel.HDFS_TSV_DESCRIPTOR)
                         .createInstance(giraphExecutor, null, -1);
-        inputChannelInstance.addPath(this.getClass().getResource("/test.edgelist").toString());
+        inputChannelInstance.addPath(this.getClass().getResource("/test.edgelist.input").toString());
         inputChannelInstance.getLineage().collectAndMark();
 
         final ExecutionOperator inputOperator = mock(ExecutionOperator.class);
diff --git a/wayang-platforms/wayang-giraph/src/test/resources/test.edgelist b/wayang-platforms/wayang-giraph/src/test/resources/test.edgelist.input
similarity index 100%
rename from wayang-platforms/wayang-giraph/src/test/resources/test.edgelist
rename to wayang-platforms/wayang-giraph/src/test/resources/test.edgelist.input
diff --git a/wayang-platforms/wayang-graphchi/src/test/java/org/apache/wayang/graphchi/operators/GraphChiPageRankOperatorTest.java b/wayang-platforms/wayang-graphchi/src/test/java/org/apache/wayang/graphchi/operators/GraphChiPageRankOperatorTest.java
index 2cf9f3a..c3d11da 100644
--- a/wayang-platforms/wayang-graphchi/src/test/java/org/apache/wayang/graphchi/operators/GraphChiPageRankOperatorTest.java
+++ b/wayang-platforms/wayang-graphchi/src/test/java/org/apache/wayang/graphchi/operators/GraphChiPageRankOperatorTest.java
@@ -69,7 +69,7 @@
         FileChannel.Instance inputChannelInstance =
                 (FileChannel.Instance) new FileChannel(FileChannel.HDFS_TSV_DESCRIPTOR)
                         .createInstance(graphChiExecutor, null, -1);
-        inputChannelInstance.addPath(this.getClass().getResource("/test.edgelist").toString());
+        inputChannelInstance.addPath(this.getClass().getResource("/test.edgelist.input").toString());
         inputChannelInstance.getLineage().collectAndMark();
 
         final ExecutionOperator inputOperator = mock(ExecutionOperator.class);
diff --git a/wayang-platforms/wayang-graphchi/src/test/resources/test.edgelist b/wayang-platforms/wayang-graphchi/src/test/resources/test.edgelist.input
similarity index 100%
rename from wayang-platforms/wayang-graphchi/src/test/resources/test.edgelist
rename to wayang-platforms/wayang-graphchi/src/test/resources/test.edgelist.input
diff --git a/wayang-platforms/wayang-java/src/test/java/org/apache/wayang/java/operators/JavaObjectFileSourceTest.java b/wayang-platforms/wayang-java/src/test/java/org/apache/wayang/java/operators/JavaObjectFileSourceTest.java
index 0a3d9d6..d5fdd55 100644
--- a/wayang-platforms/wayang-java/src/test/java/org/apache/wayang/java/operators/JavaObjectFileSourceTest.java
+++ b/wayang-platforms/wayang-java/src/test/java/org/apache/wayang/java/operators/JavaObjectFileSourceTest.java
@@ -43,7 +43,7 @@
         JavaExecutor javaExecutor = null;
         try {
             // Prepare the source.
-            final URL inputUrl = this.getClass().getResource("/0-to-10000.sequence_file");
+            final URL inputUrl = this.getClass().getResource("/0-to-10000.input");
             JavaObjectFileSource<Integer> source = new JavaObjectFileSource<>(
                     inputUrl.toString(), DataSetType.createDefault(Integer.class));
 
diff --git a/wayang-platforms/wayang-java/src/test/resources/0-to-10000.sequence_file b/wayang-platforms/wayang-java/src/test/resources/0-to-10000.input
similarity index 100%
rename from wayang-platforms/wayang-java/src/test/resources/0-to-10000.sequence_file
rename to wayang-platforms/wayang-java/src/test/resources/0-to-10000.input
Binary files differ
diff --git a/wayang-platforms/wayang-spark/src/test/java/org/apache/wayang/spark/operators/SparkObjectFileSourceTest.java b/wayang-platforms/wayang-spark/src/test/java/org/apache/wayang/spark/operators/SparkObjectFileSourceTest.java
index 30d1282..20b7dba 100644
--- a/wayang-platforms/wayang-spark/src/test/java/org/apache/wayang/spark/operators/SparkObjectFileSourceTest.java
+++ b/wayang-platforms/wayang-spark/src/test/java/org/apache/wayang/spark/operators/SparkObjectFileSourceTest.java
@@ -44,7 +44,7 @@
         try {
 
             // Prepare the source.
-            final URL inputUrl = this.getClass().getResource("/0-to-10000.sequence_file");
+            final URL inputUrl = this.getClass().getResource("/0-to-10000.input");
             SparkObjectFileSource<Integer> source = new SparkObjectFileSource<>(
                     inputUrl.toString(), DataSetType.createDefault(Integer.class));
 
diff --git a/wayang-platforms/wayang-spark/src/test/resources/0-to-10000.sequence_file b/wayang-platforms/wayang-spark/src/test/resources/0-to-10000.input
similarity index 100%
rename from wayang-platforms/wayang-spark/src/test/resources/0-to-10000.sequence_file
rename to wayang-platforms/wayang-spark/src/test/resources/0-to-10000.input
Binary files differ
diff --git a/wayang-tests-integration/src/test/java/org/apache/wayang/tests/WayangPlans.java b/wayang-tests-integration/src/test/java/org/apache/wayang/tests/WayangPlans.java
index 88ef6d7..c4a7567 100644
--- a/wayang-tests-integration/src/test/java/org/apache/wayang/tests/WayangPlans.java
+++ b/wayang-tests-integration/src/test/java/org/apache/wayang/tests/WayangPlans.java
@@ -77,15 +77,15 @@
  */
 public class WayangPlans {
 
-    public static final URI FILE_SOME_LINES_TXT = createUri("/some-lines.txt");
+    public static final URI FILE_SOME_LINES_TXT = createUri("/some-lines.input");
 
-    public static final URI FILE_OTHER_LINES_TXT = createUri("/other-lines.txt");
+    public static final URI FILE_OTHER_LINES_TXT = createUri("/other-lines.input");
 
-    public static final URI ULYSSES_TXT = createUri("/ulysses.txt");
+    public static final URI ULYSSES_TXT = createUri("/ulysses.input");
 
-    public static final URI FILE_WITH_KEY_1 = createUri("/lines-with-key1.txt");
+    public static final URI FILE_WITH_KEY_1 = createUri("/lines-with-key1.input");
 
-    public static final URI FILE_WITH_KEY_2 = createUri("/lines-with-key2.txt");
+    public static final URI FILE_WITH_KEY_2 = createUri("/lines-with-key2.input");
 
     public static URI createUri(String resourcePath) {
         try {
diff --git a/wayang-tests-integration/src/test/resources/lines-with-key1.txt b/wayang-tests-integration/src/test/resources/lines-with-key1.input
similarity index 100%
rename from wayang-tests-integration/src/test/resources/lines-with-key1.txt
rename to wayang-tests-integration/src/test/resources/lines-with-key1.input
diff --git a/wayang-tests-integration/src/test/resources/lines-with-key2.txt b/wayang-tests-integration/src/test/resources/lines-with-key2.input
similarity index 100%
rename from wayang-tests-integration/src/test/resources/lines-with-key2.txt
rename to wayang-tests-integration/src/test/resources/lines-with-key2.input
diff --git a/wayang-tests-integration/src/test/resources/other-lines.txt b/wayang-tests-integration/src/test/resources/other-lines.input
similarity index 100%
rename from wayang-tests-integration/src/test/resources/other-lines.txt
rename to wayang-tests-integration/src/test/resources/other-lines.input
diff --git a/wayang-tests-integration/src/test/resources/some-lines.txt b/wayang-tests-integration/src/test/resources/some-lines.input
similarity index 100%
rename from wayang-tests-integration/src/test/resources/some-lines.txt
rename to wayang-tests-integration/src/test/resources/some-lines.input
diff --git a/wayang-tests-integration/src/test/resources/ulysses.txt b/wayang-tests-integration/src/test/resources/ulysses.input
similarity index 100%
rename from wayang-tests-integration/src/test/resources/ulysses.txt
rename to wayang-tests-integration/src/test/resources/ulysses.input